You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by ramkrish86 <gi...@git.apache.org> on 2017/01/18 09:15:41 UTC

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

GitHub user ramkrish86 opened a pull request:

    https://github.com/apache/flink/pull/3149

    FLINK-2168 Add HBaseTableSource

    Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
    If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
    In addition to going through the list, please provide a meaningful description of your changes.
    
    - [ ] General
      - The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
      - The pull request addresses only one issue
      - Each commit in the PR has a meaningful commit message (including the JIRA id)
    
    - [ ] Documentation
      - Documentation has been added for new functionality
      - Old documentation affected by the pull request has been updated
      - JavaDoc for public methods has been added
    
    - [ ] Tests & Build
      - Functionality added by the pull request is covered by tests
      - `mvn clean verify` has been executed successfully locally or a Travis build has passed
    
    @fhueske 
    Trying to create the first version of this PR. I have made the necessary changes to support HBaseTableSource by creating a HBaseTableInputFormat but lot of code is duplicated with TableInputFormat. I have not unified them for now.
    I tried compiling this code in my linux box but the @Override that I have added in HBaseTableSource overriding the BatchTableSource API shows as compilation issues but my IntelliJ IDE is fine and does not complain.
    Pls provide your valuable feed back so that I can rebase the next PR with suitable fixes. 
    Thanks. 

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ramkrish86/flink FLINK-2168

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/3149.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #3149
    
----
commit 509c294fe64a3690d93e011aa54a9dd25302a122
Author: Ramkrishna <ra...@intel.com>
Date:   2017-01-18T06:57:23Z

    FLINK-2168 Add HBaseTableSource

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97901774
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +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.types.Row;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.conf.Configuration;
    +
    +/**
    + * 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
    + */
    +// TODO : Implement ProjectableTableSource?
    +public class HBaseTableSource implements BatchTableSource<Row> {
    --- End diff --
    
    will be `StreamTableSource` okay for HBase source?
    I think it can be implemented same as batch `getDataSet`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @ramkrish86 ProjectableTableSource works in scan process. Without it TableScan is scanning all collumns and applying flatMap function on the whole data to project and filter. It's inefficient compared with pushing a projection and filtering right inside of scan proccess


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Hi all,
    
    thanks for the feedback. Let's stick to the nested schema then.
    
    I think the best approach to support projections on nested fields is to implement a second interface (i.e., a trait without default implementation) called `NestedFieldsProjectableTableSource` as @tonycox suggested.
    Adding a method with default implementation to `ProjectableTableSource` would not work, because this would turn this class into a Java abstract class while it is an interface now.
    Using flat indicies is not a very nice solution either, IMO because it is not easy to parse.
    
    For now I'd suggest to keep the scope of the PR as it is right now. A bit more Java documentation on `HBaseTableSource` to explain how it is used would be great.
    We can implement the `NestedFieldsProjectableTableSource` and the changes to `HBaseTableSource` in a follow up issue.
    
    What do you think?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97709488
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair>> familyMap =
    +		new HashMap<String, List<Pair>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class
    +	};
    +	private static byte[] EMPTY_BYTE_ARRAY = new byte[0];
    +	public void addColumns(String family, String qualifier, TypeInformation<?> type) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(family, "qualifier name");
    +		Preconditions.checkNotNull(type, "type name");
    +		List<Pair> list = this.familyMap.get(family);
    +		if (list == null) {
    +			list = new ArrayList<Pair>();
    +		}
    +		boolean found = false;
    +		for(Class classType : CLASS_TYPES) {
    +			if(classType == type.getTypeClass()) {
    +				found = true;
    +				break;
    +			}
    +		}
    +		if(!found) {
    +			// by default it will be byte[] type only
    +			type = BasicArrayTypeInfo.BYTE_ARRAY_TYPE_INFO;
    +		}
    +		list.add(new Pair(qualifier, type));
    --- End diff --
    
    Ok


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98805799
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,196 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSchema;
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.test.util.TestBaseUtils;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +	private static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	private static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	private static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	private static final byte[] F_1 = Bytes.toBytes("f1");
    +	private static final byte[] F_2 = Bytes.toBytes("f2");
    +	private static final byte[] Q_1 = Bytes.toBytes("q1");
    +	private static final byte[] Q_2 = Bytes.toBytes("q2");
    +	private static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@BeforeClass
    +	public static void activateHBaseCluster(){
    +		registerHBaseMiniClusterInClasspath();
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithSingleColumnFamily() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test");
    +		// no split keys
    +		byte[][] famNames = new byte[1][];
    +		famNames[0] = F_1;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19992L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993L));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    --- End diff --
    
    remove spaces after `env,`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97897602
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    +		new HashMap<String, List<Pair<String, TypeInformation<?>>>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class, byte[].class
    +	};
    +	/**
    +	 * Allows specifying the family and qualifier name along with the data type of the qualifier for an HBase table
    +	 *
    +	 * @param family    the family name
    +	 * @param qualifier the qualifier name
    +	 * @param clazz     the data type of the qualifier
    +	 */
    +	public void addColumn(String family, String qualifier, Class<?> clazz) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(family, "qualifier name");
    --- End diff --
    
    must be qualifier


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97956487
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,196 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSchema;
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.test.util.TestBaseUtils;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +	private static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	private static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	private static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	private static final byte[] F_1 = Bytes.toBytes("f1");
    +	private static final byte[] F_2 = Bytes.toBytes("f2");
    +	private static final byte[] Q_1 = Bytes.toBytes("q1");
    +	private static final byte[] Q_2 = Bytes.toBytes("q2");
    +	private static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@BeforeClass
    +	public static void activateHBaseCluster(){
    +		registerHBaseMiniClusterInClasspath();
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithSingleColumnFamily() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test");
    +		// no split keys
    +		byte[][] famNames = new byte[1][];
    +		famNames[0] = F_1;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19992L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993L));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		HBaseTableSchema schema = new HBaseTableSchema();
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_3), Long.class);
    +		// fetch row2 from the table till the end
    +		BatchTableSource hbaseTable = new HBaseTableSource(getConf(), tableName.getNameAsString(), schema);
    +		tableEnv.registerTableSource("test", hbaseTable);
    +		Table result = tableEnv
    +			.sql("SELECT test.f1.q1, test.f1.q2, test.f1.q3 FROM test");
    +		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    +		List<Row> results = resultSet.collect();
    +
    +		String expected = "100,strvalue,19991\n" +
    +			"101,strvalue1,19992\n" +
    +			"102,strvalue2,19993\n";
    +		TestBaseUtils.compareResultAsText(results, expected);
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithTwoColumnFamily() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test1");
    +		// no split keys
    +		byte[][] famNames = new byte[2][];
    +		famNames[0] = F_1;
    +		famNames[1] = F_2;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_2, Q_1, Bytes.toBytes(201));
    +		//2nd qual is String
    +		put.addColumn(F_2, Q_2, Bytes.toBytes("newvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_2, Q_3, Bytes.toBytes(29992L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993L));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		HBaseTableSchema schema = new HBaseTableSchema();
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_3), Long.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_3), Long.class);
    +		// fetch row2 from the table till the end
    +		BatchTableSource hbaseTable = new HBaseTableSource(getConf(), tableName.getNameAsString(), schema);
    +		tableEnv.registerTableSource("test1", hbaseTable);
    +		Table result = tableEnv
    +			.sql("SELECT test1.f1.q1, test1.f1.q2, test1.f1.q3, test1.f2.q1, test1.f2.q2, test1.f2.q3 FROM test1");
    +		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    +		List<Row> results = resultSet.collect();
    +
    +		String expected = "100,strvalue,19991,null,null,null\n" +
    +			"null,null,null,201,newvalue1,29992\n" +
    +			"102,strvalue2,19993,null,null,null\n";
    +		TestBaseUtils.compareResult(results, expected, false, false);
    --- End diff --
    
    Please use `TestBaseUtils.compareResultAsText(results, expected);`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97710068
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair>> familyMap =
    +		new HashMap<String, List<Pair>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class
    +	};
    +	private static byte[] EMPTY_BYTE_ARRAY = new byte[0];
    +	public void addColumns(String family, String qualifier, TypeInformation<?> type) {
    --- End diff --
    
    Yes. We can use `TypeExtractor.getForClass(Class)` to extract the corresponding TypeInformation for internal use.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    I tried to compare TableInputFormat and the new one. But the interesting part is HBaseInputformat does not fail in jdk 8 which was my default in my test env.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    The `NestedFieldsProjectableTableSource` makes sense to me. +1 to implement it in a separate JIRA.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97702388
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends TableInputFormat<Row> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private transient Connection conn;
    +	private transient org.apache.hadoop.conf.Configuration conf;
    +	private HBaseTableSchema schema;
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, HBaseTableSchema schema) {
    +		this.tableName = tableName;
    +		this.conf = conf;
    +		this.schema = schema;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = getScanner();
    +		}
    +	}
    +
    +	@Override
    +	protected Scan getScanner() {
    +		// TODO : Pass 'rowkey'. For this we need FilterableTableSource
    +		Scan scan = new Scan();
    +		Map<String, List<Pair>> familyMap = schema.getFamilyMap();
    +		for(String family : familyMap.keySet()) {
    +			// select only the fields in the 'selectedFields'
    +			List<Pair> colDetails = familyMap.get(family);
    +			for(Pair<String, TypeInformation<?>> pair : colDetails) {
    +				scan.addColumn(Bytes.toBytes(family), Bytes.toBytes(pair.getFirst()));
    +			}
    +		}
    +		return scan;
    +	}
    +
    +	@Override
    +	public String getTableName() {
    +		return tableName;
    +	}
    +
    +	@Override
    +	protected Row mapResultToTuple(Result res) {
    +		List<Object> values = new ArrayList<Object>();
    +		int i = 0;
    +		Map<String, List<Pair>> familyMap = schema.getFamilyMap();
    +		Row[] rows = new Row[familyMap.size()];
    +		for(String family : familyMap.keySet()) {
    +			List<Pair> colDetails = familyMap.get(family);
    +			for(Pair<String, TypeInformation<?>> pair : colDetails) {
    +				byte[] value = res.getValue(Bytes.toBytes(family), Bytes.toBytes(pair.getFirst()));
    +				if(value != null) {
    +					values.add(schema.deserialize(value, pair.getSecond()));
    +				} else {
    +					values.add(schema.deserializeNull(pair.getSecond()));
    --- End diff --
    
    >But one main concern from my side is how to present the 'NULL' means we specify a column with a type but there is no data for that column. For now I have handled by returning the Int, Float, Long - Min_values. But that may not be right I believe. Feedback and suggestions welcome.
    
    I think we can return `null` directly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r96643785
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java ---
    @@ -0,0 +1,75 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +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;
    +
    +/**
    + * 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> {
    +
    +	private Configuration conf;
    +	private String tableName;
    +	private byte[] rowKey;
    +	private String[] colNames;
    +	private TypeInformation<?>[] colTypes;
    +
    +	public HBaseTableSource(Configuration conf, String tableName, byte[] rowKey, String[] colNames,
    +							TypeInformation<?>[] colTypes) {
    +		this.conf = conf;
    +		this.tableName = Preconditions.checkNotNull(tableName, "Table  name");
    +		this.rowKey = Preconditions.checkNotNull(rowKey, "Rowkey");
    --- End diff --
    
    What is the rowKey used for ?  I think we can remove it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98455947
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,140 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.sql.Time;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.sql.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    --- End diff --
    
    why List of Pairs?
    maybe should use Map instead?
    ```java
    Map<String, Map<String, TypeInformation<?>>> familyMap
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    I think going through the PR for https://issues.apache.org/jira/browse/FLINK-3848 - I think we try to project only the required columns. Similarly we could do here also. So my and @tonycox 's suggestion of having a new way of ProjectableTableSource could help here. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    To understand better
    
    >  We could make flat schema an optional mode or implement it as a separate TableSource as well.
    and this one
    
    > This could be solved if we use a flat schema and encode the nesting as columnFamily$column
    
    Are you talking about using seperators for it? May be am not getting your concern here. Ya I agree that nested schema is better API but if we go with flat schema then maintaining the family to qualifier relation may not be easy. As you said a seperate TableSource where we define such things would be better.
    Regarding HBaseTableSchema I think that is better so that we could modify that class for better serialization and deserialization by adding more logic for different types of classes. Even when we go with flat schema I think this type of class would help us to maintain the logic of family to qualifier mapping?



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97954282
  
    --- Diff: flink-connectors/flink-hbase/pom.xml ---
    @@ -236,6 +246,12 @@ under the License.
     			<scope>test</scope>
     			<type>test-jar</type>
     		</dependency>
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-test-utils_2.10</artifactId>
    +			<version>1.3-SNAPSHOT</version>
    --- End diff --
    
    Please use `${project.version}` here,  do not hard code the version.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @ramkrish86 java8 has different memory model, https://dzone.com/articles/java-8-permgen-metaspace


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97736554
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair>> familyMap =
    +		new HashMap<String, List<Pair>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class
    +	};
    +	private static byte[] EMPTY_BYTE_ARRAY = new byte[0];
    +	public void addColumns(String family, String qualifier, TypeInformation<?> type) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(family, "qualifier name");
    +		Preconditions.checkNotNull(type, "type name");
    +		List<Pair> list = this.familyMap.get(family);
    +		if (list == null) {
    +			list = new ArrayList<Pair>();
    +		}
    +		boolean found = false;
    +		for(Class classType : CLASS_TYPES) {
    +			if(classType == type.getTypeClass()) {
    +				found = true;
    +				break;
    +			}
    +		}
    +		if(!found) {
    +			// by default it will be byte[] type only
    +			type = BasicArrayTypeInfo.BYTE_ARRAY_TYPE_INFO;
    +		}
    +		list.add(new Pair(qualifier, type));
    +		familyMap.put(family, list);
    +	}
    +
    +	public Map<String, List<Pair>> getFamilyMap() {
    +		return this.familyMap;
    +	}
    +
    +	public Object deserialize(byte[] value, TypeInformation<?> typeInfo) {
    +		if (typeInfo.isBasicType()) {
    +			if (typeInfo.getTypeClass() == Integer.class) {
    +				return Bytes.toInt(value);
    +			} else if (typeInfo.getTypeClass() == Short.class) {
    +				return Bytes.toShort(value);
    +			} else if (typeInfo.getTypeClass() == Float.class) {
    +				return Bytes.toFloat(value);
    +			} else if (typeInfo.getTypeClass() == Long.class) {
    +				return Bytes.toLong(value);
    +			} else if (typeInfo.getTypeClass() == String.class) {
    +				return Bytes.toString(value);
    +			} else if (typeInfo.getTypeClass() == Byte.class) {
    +				return value[0];
    +			} else if (typeInfo.getTypeClass() == Boolean.class) {
    +				return Bytes.toBoolean(value);
    +			} else if (typeInfo.getTypeClass() == Double.class) {
    +				return Bytes.toDouble(value);
    +			} else if (typeInfo.getTypeClass() == BigInteger.class) {
    +				return new BigInteger(value);
    +			} else if (typeInfo.getTypeClass() == BigDecimal.class) {
    +				return Bytes.toBigDecimal(value);
    +			} else if (typeInfo.getTypeClass() == Date.class) {
    +				return new Date(Bytes.toLong(value));
    +			}
    +		}
    +		return value;
    +	}
    +
    +	public Object deserializeNull(TypeInformation<?> typeInfo) {
    +		// TODO : this may need better handling.
    +		if(typeInfo.getTypeClass() ==  Integer.class) {
    --- End diff --
    
    But returning default values may not be right. I just did that here as I was not aware of that NULL handling. When a column does not have any value then it is better we return NULL rather than any default value. @ex00 - What do you think?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @ramkrish86 @fhueske what do you think about to throw `Tuple` (`T extends Tuple`)  out of `org.apache.flink.addons.hbase.TableInputFormat` and implement this abstract class in your `HBaseTableSourceInputFormat` ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    I think the OutOfMemoryError error has nothing to do with this PR.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    I think that is not the only reason. Some where either these tests are creating more static objects or bigger objects that live for the life time of the JVM. May be this test exposes it. Actually this change in maxPermsize calls for a discussion on how the 'mvn' settings should be and what is the minimum size of heap required. Also in other projects the default jdk version is now moved to 8 atleast for the trunk version. I think similar thing can be done here. Thanks @tonycox .


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97934406
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    +		new HashMap<String, List<Pair<String, TypeInformation<?>>>>();
    --- End diff --
    
    Ok. In our other projects we used to qualify the generic on both the sides. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r96665731
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,117 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.Test;
    +import scala.tools.cmd.gen.AnyVals;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +
    +	public static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	public static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	public static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	public static final byte[] F_1 = Bytes.toBytes("f1");
    +	public static final byte[] Q_1 = Bytes.toBytes("q1");
    +	public static final byte[] Q_2 = Bytes.toBytes("q2");
    +	public static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@Test
    +	public void testHBaseTableSource() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test");
    +		createTable(tableName, F_1, new byte[1][]);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_2, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_2, Bytes.toBytes(19992l));
    --- End diff --
    
    same as above


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98805341
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends TableInputFormat<Row> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private transient Connection conn;
    +	private transient org.apache.hadoop.conf.Configuration conf;
    +	private HBaseTableSchema schema;
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, HBaseTableSchema schema) {
    +		this.tableName = tableName;
    +		this.conf = conf;
    +		this.schema = schema;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = getScanner();
    +		}
    +	}
    +
    +	@Override
    +	protected Scan getScanner() {
    +		Scan scan = new Scan();
    +		for(String family : schema.getFamilyNames()) {
    --- End diff --
    
    Is it guaranteed that `schema.getFamilyNames()` always returns the names in the same order?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    I observed an `OutOfMemoryError` in one of the Travis profiles when running the final tests. Will need to figure out what's the reason for that before I can merge the PR. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @ramkrish86 You can use jhat to read hprof dump file. 
    If set `-XX:MaxPermSize=128m`
    ```xml
    <plugin>
    	<groupId>org.apache.maven.plugins</groupId>
    	<artifactId>maven-surefire-plugin</artifactId>
    	<version>2.19.1</version>
    	<configuration>
    		<argLine>-XX:MaxPermSize=128m</argLine>
    		<!-- Enforce single fork execution due to heavy mini cluster use in the tests -->
    		<forkCount>1</forkCount>
    	</configuration>
    </plugin>
    ```
    all tests will pass



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Hi @fhueske , 
    
    Regarding to the field type serialization, I think maybe we can provide default deserialization for basic types (int,long,String...) if users do use the `Bytes.toBytes(...)` to serialize the basic types. If not, users can ask this field to return raw bytes in this way:  `htableSchame.add("column_family", "qualifier", byte[].class)` and then use an user defined scalar function to deserialize the value.
    
    regarding to the rowkeys, I agree with you. It would be great if we can set scan range by WHERE clause. But FLINK-3849 (FilterableTableSource) is still a pending PR, I would suggest to break this issue into two. 1. add HBaseTableSource, provide access to HBase tables and support nested schema. 2. extend HBaseTableSource to support FilterableTableSource.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r96665755
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,117 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.Test;
    +import scala.tools.cmd.gen.AnyVals;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +
    +	public static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	public static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	public static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	public static final byte[] F_1 = Bytes.toBytes("f1");
    +	public static final byte[] Q_1 = Bytes.toBytes("q1");
    +	public static final byte[] Q_2 = Bytes.toBytes("q2");
    +	public static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@Test
    +	public void testHBaseTableSource() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test");
    +		createTable(tableName, F_1, new byte[1][]);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_2, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_2, Bytes.toBytes(19992l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_2, Bytes.toBytes(19993l));
    --- End diff --
    
    same as above


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @fhueske - Are you fine with that pom change? If so we can get this in.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r96668950
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
    +import org.apache.flink.api.common.io.RichInputFormat;
    +import org.apache.flink.api.common.io.statistics.BaseStatistics;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.io.InputSplitAssigner;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.client.Table;
    +import org.apache.hadoop.hbase.client.ClusterConnection;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.ResultScanner;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.HRegionLocator;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends RichInputFormat<Row, TableInputSplit> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private TypeInformation[] fieldTypeInfos;
    +	private String[] fieldNames;
    +	private transient Table table;
    +	private transient Scan scan;
    +	private transient Connection conn;
    +	private ResultScanner resultScanner = null;
    +
    +	private byte[] lastRow;
    +	private int scannedRows;
    +	private boolean endReached = false;
    +	private org.apache.hadoop.conf.Configuration conf;
    +	private static final String COLON = ":";
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, String[] fieldNames, TypeInformation[] fieldTypeInfos) {
    +		this.conf = conf;
    +		this.tableName = tableName;
    +		this.fieldNames = fieldNames;
    +		this.fieldTypeInfos = fieldTypeInfos;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = createScanner();
    +		}
    +	}
    +
    +	private Scan createScanner() {
    +		Scan scan = new Scan();
    +		for(String field : fieldNames) {
    +			// select only the fields in the 'selectedFields'
    +			String[] famCol = field.split(COLON);
    +			scan.addColumn(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +		}
    +		return scan;
    +	}
    +
    +	private void connectToTable() {
    +		//use files found in the classpath
    +		if(this.conf == null) {
    +			this.conf = HBaseConfiguration.create();
    +		}
    +		try {
    +			conn = ConnectionFactory.createConnection(this.conf);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while creating connection to hbase cluster", ioe);
    +			return;
    +		}
    +		try {
    +			table = conn.getTable(TableName.valueOf(tableName));
    +		} catch(TableNotFoundException tnfe) {
    +			LOG.error("The table " + tableName + " not found ", tnfe);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while connecting to the table "+tableName+ " ", ioe);
    +		}
    +	}
    +
    +	@Override
    +	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
    +		return null;
    +	}
    +
    +	@Override
    +	public TableInputSplit[] createInputSplits(final int minNumSplits) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +
    +		//Gets the starting and ending row keys for every region in the currently open table
    +		HRegionLocator regionLocator = new HRegionLocator(table.getName(), (ClusterConnection) conn);
    +		final Pair<byte[][], byte[][]> keys = regionLocator.getStartEndKeys();
    +		if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
    +			throw new IOException("Expecting at least one region.");
    +		}
    +		final byte[] startRow = scan.getStartRow();
    +		final byte[] stopRow = scan.getStopRow();
    +		final boolean scanWithNoLowerBound = startRow.length == 0;
    +		final boolean scanWithNoUpperBound = stopRow.length == 0;
    +
    +		final List<TableInputSplit> splits = new ArrayList<TableInputSplit>(minNumSplits);
    +		for (int i = 0; i < keys.getFirst().length; i++) {
    +			final byte[] startKey = keys.getFirst()[i];
    +			final byte[] endKey = keys.getSecond()[i];
    +			final String regionLocation = regionLocator.getRegionLocation(startKey, false).getHostnamePort();
    +			//Test if the given region is to be included in the InputSplit while splitting the regions of a table
    +			if (!includeRegionInSplit(startKey, endKey)) {
    +				continue;
    +			}
    +			//Finds the region on which the given row is being served
    +			final String[] hosts = new String[]{regionLocation};
    +
    +			// determine if regions contains keys used by the scan
    +			boolean isLastRegion = endKey.length == 0;
    +			if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0) &&
    +				(scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) {
    +
    +				final byte[] splitStart = scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0 ? startKey : startRow;
    +				final byte[] splitStop = (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0)
    +					&& !isLastRegion ? endKey : stopRow;
    +				int id = splits.size();
    +				final TableInputSplit split = new TableInputSplit(id, hosts, table.getName().getName(), splitStart, splitStop);
    +				splits.add(split);
    +			}
    +		}
    +		LOG.info("Created " + splits.size() + " splits");
    +		for (TableInputSplit split : splits) {
    +			logSplitInfo("created", split);
    +		}
    +		return splits.toArray(new TableInputSplit[0]);
    +	}
    +
    +	protected boolean includeRegionInSplit(final byte[] startKey, final byte[] endKey) {
    +		return true;
    +	}
    +
    +	@Override
    +	public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
    +		return new LocatableInputSplitAssigner(inputSplits);
    +	}
    +
    +	@Override
    +	public void open(TableInputSplit split) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +		if (split == null) {
    +			throw new IOException("Input split is null!");
    +		}
    +
    +		logSplitInfo("opening", split);
    +		// set the start row and stop row from the splits
    +		scan.setStartRow(split.getStartRow());
    +		lastRow = split.getEndRow();
    +		scan.setStopRow(lastRow);
    +
    +		resultScanner = table.getScanner(scan);
    +		endReached = false;
    +		scannedRows = 0;
    +	}
    +
    +	private void logSplitInfo(String action, TableInputSplit split) {
    +		int splitId = split.getSplitNumber();
    +		String splitStart = Bytes.toString(split.getStartRow());
    +		String splitEnd = Bytes.toString(split.getEndRow());
    +		String splitStartKey = splitStart.isEmpty() ? "-" : splitStart;
    +		String splitStopKey = splitEnd.isEmpty() ? "-" : splitEnd;
    +		String[] hostnames = split.getHostnames();
    +		LOG.info("{} split (this={})[{}|{}|{}|{}]", action, this, splitId, hostnames, splitStartKey, splitStopKey);
    +	}
    +
    +	@Override
    +	public boolean reachedEnd() throws IOException {
    +		return endReached;
    +	}
    +
    +	@Override
    +	public Row nextRecord(Row reuse) throws IOException {
    +		if (resultScanner == null) {
    +			throw new IOException("No table result scanner provided!");
    +		}
    +		try {
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		} catch (Exception e) {
    +			resultScanner.close();
    +			//workaround for timeout on scan
    +			LOG.warn("Error after scan of " + scannedRows + " rows. Retry with a new scanner...", e);
    +			scan.setStartRow(lastRow);
    +			resultScanner = table.getScanner(scan);
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		}
    +		endReached = true;
    +		return null;
    +	}
    +
    +	private Row mapResultToRow(Result res) {
    +		Object[] values = new Object[fieldNames.length];
    +		int i = 0;
    +		for(String field : fieldNames) {
    +			String[] famCol = field.split(COLON);
    +			byte[] value = res.getValue(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +			TypeInformation typeInfo = fieldTypeInfos[i];
    +			if(typeInfo.isBasicType()) {
    +				if(typeInfo.getTypeClass() ==  Integer.class) {
    +					values[i] = Bytes.toInt(value);
    +				} else if(typeInfo.getTypeClass() == Short.class) {
    +					values[i] = Bytes.toShort(value);
    +				} else if(typeInfo.getTypeClass() == Float.class) {
    +					values[i] = Bytes.toFloat(value);
    +				} else if(typeInfo.getTypeClass() == Long.class) {
    +					values[i] = Bytes.toLong(value);
    +				} else if(typeInfo.getTypeClass() == String.class) {
    +					values[i] = Bytes.toString(value);
    +				} else if(typeInfo.getTypeClass() == Byte.class) {
    +					values[i] = value[0];
    +				} else if(typeInfo.getTypeClass() == Boolean.class) {
    +					values[i] = Bytes.toBoolean(value);
    +				} else if(typeInfo.getTypeClass() == Double.class) {
    +					values[i] = Bytes.toDouble(value);
    +				} else if(typeInfo.getTypeClass() == BigInteger.class) {
    +					values[i] = new BigInteger(value);
    +				} else if(typeInfo.getTypeClass() == BigDecimal.class) {
    +					values[i] = Bytes.toBigDecimal(value);
    +				} else if(typeInfo.getTypeClass() == Date.class) {
    +					values[i] = new Date(Bytes.toLong(value));
    +				}
    +			} else {
    +				// TODO for other types??
    --- End diff --
    
    Arrays, Java SQL Date/Time/Timestamp, CompositeType


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    > Will need to figure out what's the reason for that before I can merge the PR.
    
    I tried running those tests again in my linux box and all went through without any error.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97698410
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair>> familyMap =
    +		new HashMap<String, List<Pair>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class
    +	};
    +	private static byte[] EMPTY_BYTE_ARRAY = new byte[0];
    +	public void addColumns(String family, String qualifier, TypeInformation<?> type) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(family, "qualifier name");
    +		Preconditions.checkNotNull(type, "type name");
    +		List<Pair> list = this.familyMap.get(family);
    +		if (list == null) {
    +			list = new ArrayList<Pair>();
    +		}
    +		boolean found = false;
    +		for(Class classType : CLASS_TYPES) {
    +			if(classType == type.getTypeClass()) {
    +				found = true;
    +				break;
    +			}
    +		}
    +		if(!found) {
    +			// by default it will be byte[] type only
    +			type = BasicArrayTypeInfo.BYTE_ARRAY_TYPE_INFO;
    +		}
    +		list.add(new Pair(qualifier, type));
    --- End diff --
    
    add a `<>` to avoid warning. `list.add(new Pair<>(qualifier, type));`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97696231
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair>> familyMap =
    +		new HashMap<String, List<Pair>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class
    +	};
    +	private static byte[] EMPTY_BYTE_ARRAY = new byte[0];
    +	public void addColumns(String family, String qualifier, TypeInformation<?> type) {
    --- End diff --
    
    addColumns -> addColumn ?
    
    I'm not sure whether we should use `TypeInformation` or `Class` here. Because TypeInformation indicates that we use Flink Serialization framework to serialize primitives. But it doesn't happen here.  Maybe `Class` is a simpler and better choice ? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r96824021
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
    +import org.apache.flink.api.common.io.RichInputFormat;
    +import org.apache.flink.api.common.io.statistics.BaseStatistics;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.io.InputSplitAssigner;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.client.Table;
    +import org.apache.hadoop.hbase.client.ClusterConnection;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.ResultScanner;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.HRegionLocator;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends RichInputFormat<Row, TableInputSplit> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private TypeInformation[] fieldTypeInfos;
    +	private String[] fieldNames;
    +	private transient Table table;
    +	private transient Scan scan;
    +	private transient Connection conn;
    +	private ResultScanner resultScanner = null;
    +
    +	private byte[] lastRow;
    +	private int scannedRows;
    +	private boolean endReached = false;
    +	private org.apache.hadoop.conf.Configuration conf;
    +	private static final String COLON = ":";
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, String[] fieldNames, TypeInformation[] fieldTypeInfos) {
    +		this.conf = conf;
    +		this.tableName = tableName;
    +		this.fieldNames = fieldNames;
    +		this.fieldTypeInfos = fieldTypeInfos;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = createScanner();
    +		}
    +	}
    +
    +	private Scan createScanner() {
    +		Scan scan = new Scan();
    +		for(String field : fieldNames) {
    +			// select only the fields in the 'selectedFields'
    +			String[] famCol = field.split(COLON);
    +			scan.addColumn(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +		}
    +		return scan;
    +	}
    +
    +	private void connectToTable() {
    +		//use files found in the classpath
    +		if(this.conf == null) {
    +			this.conf = HBaseConfiguration.create();
    +		}
    +		try {
    +			conn = ConnectionFactory.createConnection(this.conf);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while creating connection to hbase cluster", ioe);
    +			return;
    +		}
    +		try {
    +			table = conn.getTable(TableName.valueOf(tableName));
    +		} catch(TableNotFoundException tnfe) {
    +			LOG.error("The table " + tableName + " not found ", tnfe);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while connecting to the table "+tableName+ " ", ioe);
    +		}
    +	}
    +
    +	@Override
    +	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
    +		return null;
    +	}
    +
    +	@Override
    +	public TableInputSplit[] createInputSplits(final int minNumSplits) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +
    +		//Gets the starting and ending row keys for every region in the currently open table
    +		HRegionLocator regionLocator = new HRegionLocator(table.getName(), (ClusterConnection) conn);
    +		final Pair<byte[][], byte[][]> keys = regionLocator.getStartEndKeys();
    +		if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
    +			throw new IOException("Expecting at least one region.");
    +		}
    +		final byte[] startRow = scan.getStartRow();
    +		final byte[] stopRow = scan.getStopRow();
    +		final boolean scanWithNoLowerBound = startRow.length == 0;
    +		final boolean scanWithNoUpperBound = stopRow.length == 0;
    +
    +		final List<TableInputSplit> splits = new ArrayList<TableInputSplit>(minNumSplits);
    +		for (int i = 0; i < keys.getFirst().length; i++) {
    +			final byte[] startKey = keys.getFirst()[i];
    +			final byte[] endKey = keys.getSecond()[i];
    +			final String regionLocation = regionLocator.getRegionLocation(startKey, false).getHostnamePort();
    +			//Test if the given region is to be included in the InputSplit while splitting the regions of a table
    +			if (!includeRegionInSplit(startKey, endKey)) {
    +				continue;
    +			}
    +			//Finds the region on which the given row is being served
    +			final String[] hosts = new String[]{regionLocation};
    +
    +			// determine if regions contains keys used by the scan
    +			boolean isLastRegion = endKey.length == 0;
    +			if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0) &&
    +				(scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) {
    +
    +				final byte[] splitStart = scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0 ? startKey : startRow;
    +				final byte[] splitStop = (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0)
    +					&& !isLastRegion ? endKey : stopRow;
    +				int id = splits.size();
    +				final TableInputSplit split = new TableInputSplit(id, hosts, table.getName().getName(), splitStart, splitStop);
    +				splits.add(split);
    +			}
    +		}
    +		LOG.info("Created " + splits.size() + " splits");
    +		for (TableInputSplit split : splits) {
    +			logSplitInfo("created", split);
    +		}
    +		return splits.toArray(new TableInputSplit[0]);
    +	}
    +
    +	protected boolean includeRegionInSplit(final byte[] startKey, final byte[] endKey) {
    +		return true;
    +	}
    +
    +	@Override
    +	public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
    +		return new LocatableInputSplitAssigner(inputSplits);
    +	}
    +
    +	@Override
    +	public void open(TableInputSplit split) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +		if (split == null) {
    +			throw new IOException("Input split is null!");
    +		}
    +
    +		logSplitInfo("opening", split);
    +		// set the start row and stop row from the splits
    +		scan.setStartRow(split.getStartRow());
    +		lastRow = split.getEndRow();
    +		scan.setStopRow(lastRow);
    +
    +		resultScanner = table.getScanner(scan);
    +		endReached = false;
    +		scannedRows = 0;
    +	}
    +
    +	private void logSplitInfo(String action, TableInputSplit split) {
    +		int splitId = split.getSplitNumber();
    +		String splitStart = Bytes.toString(split.getStartRow());
    +		String splitEnd = Bytes.toString(split.getEndRow());
    +		String splitStartKey = splitStart.isEmpty() ? "-" : splitStart;
    +		String splitStopKey = splitEnd.isEmpty() ? "-" : splitEnd;
    +		String[] hostnames = split.getHostnames();
    +		LOG.info("{} split (this={})[{}|{}|{}|{}]", action, this, splitId, hostnames, splitStartKey, splitStopKey);
    +	}
    +
    +	@Override
    +	public boolean reachedEnd() throws IOException {
    +		return endReached;
    +	}
    +
    +	@Override
    +	public Row nextRecord(Row reuse) throws IOException {
    +		if (resultScanner == null) {
    +			throw new IOException("No table result scanner provided!");
    +		}
    +		try {
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		} catch (Exception e) {
    +			resultScanner.close();
    +			//workaround for timeout on scan
    +			LOG.warn("Error after scan of " + scannedRows + " rows. Retry with a new scanner...", e);
    +			scan.setStartRow(lastRow);
    +			resultScanner = table.getScanner(scan);
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		}
    +		endReached = true;
    +		return null;
    +	}
    +
    +	private Row mapResultToRow(Result res) {
    +		Object[] values = new Object[fieldNames.length];
    +		int i = 0;
    +		for(String field : fieldNames) {
    +			String[] famCol = field.split(COLON);
    +			byte[] value = res.getValue(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +			TypeInformation typeInfo = fieldTypeInfos[i];
    +			if(typeInfo.isBasicType()) {
    +				if(typeInfo.getTypeClass() ==  Integer.class) {
    +					values[i] = Bytes.toInt(value);
    +				} else if(typeInfo.getTypeClass() == Short.class) {
    +					values[i] = Bytes.toShort(value);
    +				} else if(typeInfo.getTypeClass() == Float.class) {
    +					values[i] = Bytes.toFloat(value);
    +				} else if(typeInfo.getTypeClass() == Long.class) {
    +					values[i] = Bytes.toLong(value);
    +				} else if(typeInfo.getTypeClass() == String.class) {
    +					values[i] = Bytes.toString(value);
    +				} else if(typeInfo.getTypeClass() == Byte.class) {
    +					values[i] = value[0];
    +				} else if(typeInfo.getTypeClass() == Boolean.class) {
    +					values[i] = Bytes.toBoolean(value);
    +				} else if(typeInfo.getTypeClass() == Double.class) {
    +					values[i] = Bytes.toDouble(value);
    +				} else if(typeInfo.getTypeClass() == BigInteger.class) {
    +					values[i] = new BigInteger(value);
    +				} else if(typeInfo.getTypeClass() == BigDecimal.class) {
    +					values[i] = Bytes.toBigDecimal(value);
    +				} else if(typeInfo.getTypeClass() == Date.class) {
    +					values[i] = new Date(Bytes.toLong(value));
    +				}
    +			} else {
    +				// TODO for other types??
    +			}
    +			i++;
    +		}
    +		return Row.of(values);
    +	}
    +
    +	@Override
    +	public void close() throws IOException {
    +		LOG.info("Closing split (scanned {} rows)", scannedRows);
    +		lastRow = null;
    +		try {
    +			if (resultScanner != null) {
    +				resultScanner.close();
    +			}
    +		} finally {
    +			resultScanner = null;
    +		}
    +	}
    +
    +	@Override
    +	public void closeInputFormat() throws IOException {
    +		try {
    +			if (table != null) {
    +				table.close();
    +			}
    +		} finally {
    +			table = null;
    +		}
    +	}
    +
    +	@Override
    +	public TypeInformation<Row> getProducedType() {
    +		return new RowTypeInfo(this.fieldTypeInfos);
    --- End diff --
    
    Using `RowTypeInfo(TypeInformation<?>[] types, String[] fieldNames)` constructor to set the custom field names.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97934537
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,248 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSchema;
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +	public static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	public static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	public static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	public static final byte[] F_1 = Bytes.toBytes("f1");
    +	public static final byte[] F_2 = Bytes.toBytes("f2");
    +	public static final byte[] Q_1 = Bytes.toBytes("q1");
    +	public static final byte[] Q_2 = Bytes.toBytes("q2");
    +	public static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@BeforeClass
    +	public static void activateHBaseCluster(){
    +		registerHBaseMiniClusterInClasspath();
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithSingleColumnFamily() throws Exception {
    +		// create a table with single region
    +		MapFunction<Row, String> mapFunction = new MapFunction<Row, String>() {
    +
    +			@Override
    +			public String map(Row value) throws Exception {
    +				return value == null ? "null" : value.toString();
    +			}
    +		};
    +		TableName tableName = TableName.valueOf("test");
    +		// no split keys
    +		byte[][] famNames = new byte[1][];
    +		famNames[0] = F_1;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19992l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993l));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		HBaseTableSchema schema = new HBaseTableSchema();
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_3), Long.class);
    +		// fetch row2 from the table till the end
    +		BatchTableSource hbaseTable = new HBaseTableSource(getConf(), tableName.getNameAsString(), schema);
    +		tableEnv.registerTableSource("test", hbaseTable);
    +		Table result = tableEnv
    +			.sql("SELECT test.f1.q1, test.f1.q2, test.f1.q3 FROM test");
    +		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    +		List<Row> results = resultSet.collect();
    +
    +		String expected = "100,strvalue,19991\n" +
    +			"101,strvalue1,19992\n" +
    +			"102,strvalue2,19993\n";
    +		compareResult(results, expected, false, true);
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithTwoColumnFamily() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test1");
    +		// no split keys
    +		byte[][] famNames = new byte[2][];
    +		famNames[0] = F_1;
    +		famNames[1] = F_2;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_2, Q_1, Bytes.toBytes(201));
    +		//2nd qual is String
    +		put.addColumn(F_2, Q_2, Bytes.toBytes("newvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_2, Q_3, Bytes.toBytes(29992l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993l));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		HBaseTableSchema schema = new HBaseTableSchema();
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_3), Long.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_3), Long.class);
    +		// fetch row2 from the table till the end
    +		BatchTableSource hbaseTable = new HBaseTableSource(getConf(), tableName.getNameAsString(), schema);
    +		tableEnv.registerTableSource("test1", hbaseTable);
    +		Table result = tableEnv
    +			.sql("SELECT test1.f1.q1, test1.f1.q2, test1.f1.q3, test1.f2.q1, test1.f2.q2, test1.f2.q3 FROM test1");
    +		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    +		List<Row> results = resultSet.collect();
    +
    +		String expected = "100,strvalue,19991,null,null,null\n" +
    +			"null,null,null,201,newvalue1,29992\n" +
    +			"102,strvalue2,19993,null,null,null\n";
    +		compareResult(results, expected, false, false);
    +	}
    +
    +
    +	static <T> void compareResult(List<T> result, String expected, boolean asTuples, boolean sort) {
    --- End diff --
    
    I did not want to make the pom change and hence did this way.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97900030
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    +		new HashMap<String, List<Pair<String, TypeInformation<?>>>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    --- End diff --
    
    I think, if it's used only in table api would be good java.sql.Date
    cause calcite use sql type of date


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @ramkrish86 I think you need implement projectable interface in HBase source


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97709503
  
    --- 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()]);
    --- End diff --
    
    Fine. Will do it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97709469
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair>> familyMap =
    +		new HashMap<String, List<Pair>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class
    +	};
    +	private static byte[] EMPTY_BYTE_ARRAY = new byte[0];
    +	public void addColumns(String family, String qualifier, TypeInformation<?> type) {
    --- End diff --
    
    Ok. Let me check that. So if we pass Class there we could wrap it with the corresponding TypeInformation for our internal usage?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ex00 <gi...@git.apache.org>.
Github user ex00 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97734069
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair>> familyMap =
    +		new HashMap<String, List<Pair>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class
    +	};
    +	private static byte[] EMPTY_BYTE_ARRAY = new byte[0];
    +	public void addColumns(String family, String qualifier, TypeInformation<?> type) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(family, "qualifier name");
    +		Preconditions.checkNotNull(type, "type name");
    +		List<Pair> list = this.familyMap.get(family);
    +		if (list == null) {
    +			list = new ArrayList<Pair>();
    +		}
    +		boolean found = false;
    +		for(Class classType : CLASS_TYPES) {
    +			if(classType == type.getTypeClass()) {
    +				found = true;
    +				break;
    +			}
    +		}
    +		if(!found) {
    +			// by default it will be byte[] type only
    +			type = BasicArrayTypeInfo.BYTE_ARRAY_TYPE_INFO;
    +		}
    +		list.add(new Pair(qualifier, type));
    +		familyMap.put(family, list);
    +	}
    +
    +	public Map<String, List<Pair>> getFamilyMap() {
    +		return this.familyMap;
    +	}
    +
    +	public Object deserialize(byte[] value, TypeInformation<?> typeInfo) {
    +		if (typeInfo.isBasicType()) {
    +			if (typeInfo.getTypeClass() == Integer.class) {
    +				return Bytes.toInt(value);
    +			} else if (typeInfo.getTypeClass() == Short.class) {
    +				return Bytes.toShort(value);
    +			} else if (typeInfo.getTypeClass() == Float.class) {
    +				return Bytes.toFloat(value);
    +			} else if (typeInfo.getTypeClass() == Long.class) {
    +				return Bytes.toLong(value);
    +			} else if (typeInfo.getTypeClass() == String.class) {
    +				return Bytes.toString(value);
    +			} else if (typeInfo.getTypeClass() == Byte.class) {
    +				return value[0];
    +			} else if (typeInfo.getTypeClass() == Boolean.class) {
    +				return Bytes.toBoolean(value);
    +			} else if (typeInfo.getTypeClass() == Double.class) {
    +				return Bytes.toDouble(value);
    +			} else if (typeInfo.getTypeClass() == BigInteger.class) {
    +				return new BigInteger(value);
    +			} else if (typeInfo.getTypeClass() == BigDecimal.class) {
    +				return Bytes.toBigDecimal(value);
    +			} else if (typeInfo.getTypeClass() == Date.class) {
    +				return new Date(Bytes.toLong(value));
    +			}
    +		}
    +		return value;
    +	}
    +
    +	public Object deserializeNull(TypeInformation<?> typeInfo) {
    +		// TODO : this may need better handling.
    +		if(typeInfo.getTypeClass() ==  Integer.class) {
    --- End diff --
    
    `Row` type isn't basic type. I think may add check `typeInfo.isBasicType()` in this method like as in `#deserialize`.  and for not primitive types could return other default values.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    I see. Am not sure how I missed that because my IDE I thought was already updated with latest code. Will check it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/3149


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    So am not sure if we can add specific API in table.api for hbase? Is that allowed?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97701834
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends TableInputFormat<Row> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private transient Connection conn;
    +	private transient org.apache.hadoop.conf.Configuration conf;
    +	private HBaseTableSchema schema;
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, HBaseTableSchema schema) {
    +		this.tableName = tableName;
    +		this.conf = conf;
    +		this.schema = schema;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = getScanner();
    +		}
    +	}
    +
    +	@Override
    +	protected Scan getScanner() {
    +		// TODO : Pass 'rowkey'. For this we need FilterableTableSource
    +		Scan scan = new Scan();
    +		Map<String, List<Pair>> familyMap = schema.getFamilyMap();
    +		for(String family : familyMap.keySet()) {
    +			// select only the fields in the 'selectedFields'
    +			List<Pair> colDetails = familyMap.get(family);
    +			for(Pair<String, TypeInformation<?>> pair : colDetails) {
    +				scan.addColumn(Bytes.toBytes(family), Bytes.toBytes(pair.getFirst()));
    +			}
    +		}
    +		return scan;
    +	}
    +
    +	@Override
    +	public String getTableName() {
    +		return tableName;
    +	}
    +
    +	@Override
    +	protected Row mapResultToTuple(Result res) {
    +		List<Object> values = new ArrayList<Object>();
    +		int i = 0;
    +		Map<String, List<Pair>> familyMap = schema.getFamilyMap();
    +		Row[] rows = new Row[familyMap.size()];
    +		for(String family : familyMap.keySet()) {
    +			List<Pair> colDetails = familyMap.get(family);
    +			for(Pair<String, TypeInformation<?>> pair : colDetails) {
    +				byte[] value = res.getValue(Bytes.toBytes(family), Bytes.toBytes(pair.getFirst()));
    +				if(value != null) {
    +					values.add(schema.deserialize(value, pair.getSecond()));
    +				} else {
    +					values.add(schema.deserializeNull(pair.getSecond()));
    --- End diff --
    
    Do we really need this method to indicate `null` using the special default value? Why not set null `values.add(null)` directly ? 
    
    `Row` supports nullable field, but Tuple doesn't.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @tonycox is right. Please rebase your PR to the current master branch. The `TableSource` interface was recently modified.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97934502
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,248 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSchema;
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +	public static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	public static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	public static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	public static final byte[] F_1 = Bytes.toBytes("f1");
    +	public static final byte[] F_2 = Bytes.toBytes("f2");
    +	public static final byte[] Q_1 = Bytes.toBytes("q1");
    +	public static final byte[] Q_2 = Bytes.toBytes("q2");
    +	public static final byte[] Q_3 = Bytes.toBytes("q3");
    --- End diff --
    
    ya . My bad. Will remove.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Thanks Robert! 
    I'll let the test run again and will merge if everything passes. Thanks @tonycox and @ramkrish86 for digging into this.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97073974
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java ---
    @@ -0,0 +1,75 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +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;
    +
    +/**
    + * 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> {
    +
    +	private Configuration conf;
    +	private String tableName;
    +	private byte[] rowKey;
    +	private String[] colNames;
    +	private TypeInformation<?>[] colTypes;
    +
    +	public HBaseTableSource(Configuration conf, String tableName, byte[] rowKey, String[] colNames,
    +							TypeInformation<?>[] colTypes) {
    +		this.conf = conf;
    +		this.tableName = Preconditions.checkNotNull(tableName, "Table  name");
    +		this.rowKey = Preconditions.checkNotNull(rowKey, "Rowkey");
    --- End diff --
    
    Hi @ramkrish86 , the RowTypeInfo contains field types and also field names (default f1~fn) of the table. By using `RowTypeInfo(TypeInformation[] types, String[] fieldNames)` constructor, we can customize the field names of the table. So in your case, you can query columns by `.sql(SELECT person.name, person.age FROM test)`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97701594
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -19,99 +19,113 @@
     package org.apache.flink.addons.hbase;
     
     import org.apache.flink.api.common.io.InputFormat;
    -import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
    -import org.apache.flink.api.common.io.RichInputFormat;
    -import org.apache.flink.api.common.io.statistics.BaseStatistics;
     import org.apache.flink.api.common.typeinfo.TypeInformation;
     import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
     import org.apache.flink.api.java.typeutils.RowTypeInfo;
     import org.apache.flink.configuration.Configuration;
    -import org.apache.flink.core.io.InputSplitAssigner;
     import org.apache.flink.types.Row;
     import org.apache.hadoop.hbase.HBaseConfiguration;
     import org.apache.hadoop.hbase.TableName;
     import org.apache.hadoop.hbase.TableNotFoundException;
    -import org.apache.hadoop.hbase.client.Scan;
    -import org.apache.hadoop.hbase.client.Table;
    -import org.apache.hadoop.hbase.client.ClusterConnection;
    -import org.apache.hadoop.hbase.client.Result;
    -import org.apache.hadoop.hbase.client.ResultScanner;
    -import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.HTable;
     import org.apache.hadoop.hbase.client.Connection;
    -import org.apache.hadoop.hbase.client.HRegionLocator;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.Scan;
     import org.apache.hadoop.hbase.util.Bytes;
     import org.apache.hadoop.hbase.util.Pair;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.math.BigDecimal;
    -import java.math.BigInteger;
     import java.util.ArrayList;
    -import java.util.Date;
     import java.util.List;
    +import java.util.Map;
     
     /**
      * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
      */
    -public class HBaseTableSourceInputFormat extends RichInputFormat<Row, TableInputSplit> implements ResultTypeQueryable<Row> {
    +public class HBaseTableSourceInputFormat extends TableInputFormat<Row> implements ResultTypeQueryable<Row> {
     
     	private static final long serialVersionUID = 1L;
     
     	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
     	private String tableName;
    -	private TypeInformation[] fieldTypeInfos;
    -	private String[] fieldNames;
    -	private transient Table table;
    -	private transient Scan scan;
     	private transient Connection conn;
    -	private ResultScanner resultScanner = null;
    -
    -	private byte[] lastRow;
    -	private int scannedRows;
    -	private boolean endReached = false;
    -	private org.apache.hadoop.conf.Configuration conf;
    -	private static final String COLON = ":";
    +	private transient org.apache.hadoop.conf.Configuration conf;
    +	private HBaseTableSchema schema;
     
    -	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, String[] fieldNames, TypeInformation[] fieldTypeInfos) {
    -		this.conf = conf;
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, HBaseTableSchema schema) {
     		this.tableName = tableName;
    -		this.fieldNames = fieldNames;
    -		this.fieldTypeInfos = fieldTypeInfos;
    +		this.conf = conf;
    +		this.schema = schema;
     	}
     
     	@Override
     	public void configure(Configuration parameters) {
     		LOG.info("Initializing HBaseConfiguration");
     		connectToTable();
     		if(table != null) {
    -			scan = createScanner();
    +			scan = getScanner();
     		}
     	}
     
    -	private Scan createScanner() {
    +	@Override
    +	protected Scan getScanner() {
    +		// TODO : Pass 'rowkey'. For this we need FilterableTableSource
     		Scan scan = new Scan();
    -		for(String field : fieldNames) {
    +		Map<String, List<Pair>> familyMap = schema.getFamilyMap();
    +		for(String family : familyMap.keySet()) {
     			// select only the fields in the 'selectedFields'
    -			String[] famCol = field.split(COLON);
    -			scan.addColumn(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +			List<Pair> colDetails = familyMap.get(family);
    +			for(Pair<String, TypeInformation<?>> pair : colDetails) {
    +				scan.addColumn(Bytes.toBytes(family), Bytes.toBytes(pair.getFirst()));
    +			}
     		}
     		return scan;
     	}
     
    +	@Override
    +	public String getTableName() {
    +		return tableName;
    +	}
    +
    +	@Override
    +	protected Row mapResultToTuple(Result res) {
    +		List<Object> values = new ArrayList<Object>();
    +		int i = 0;
    +		Map<String, List<Pair>> familyMap = schema.getFamilyMap();
    +		Row[] rows = new Row[familyMap.size()];
    --- End diff --
    
    Better to declare `rows` as `Object[]` to avoid confusing whether `rows` is a varargs or non-varargs.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    You are right @wuchong, we should break it down into two issues. 
    
    I also agree about the serialization. We should offer defaults for primitives (`byte`, `short`, `int`, `long`, `boolean`, `float`, `double`) and a set of common character encodings (UTF-8, ASCII, etc.) for `String`. Everything else can be initially handled as `byte[]`, IMO.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97934495
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +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.types.Row;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.conf.Configuration;
    +
    +/**
    + * 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
    + */
    +// TODO : Implement ProjectableTableSource?
    +public class HBaseTableSource implements BatchTableSource<Row> {
    --- End diff --
    
    Am not sure.. For now I think we will implement BatchTableSource only and later implement StreamTableSource? Is there any significant design expectation for a source to be StreamTableSource?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97936002
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,248 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSchema;
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +	public static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	public static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	public static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	public static final byte[] F_1 = Bytes.toBytes("f1");
    +	public static final byte[] F_2 = Bytes.toBytes("f2");
    +	public static final byte[] Q_1 = Bytes.toBytes("q1");
    +	public static final byte[] Q_2 = Bytes.toBytes("q2");
    +	public static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@BeforeClass
    +	public static void activateHBaseCluster(){
    +		registerHBaseMiniClusterInClasspath();
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithSingleColumnFamily() throws Exception {
    +		// create a table with single region
    +		MapFunction<Row, String> mapFunction = new MapFunction<Row, String>() {
    +
    +			@Override
    +			public String map(Row value) throws Exception {
    +				return value == null ? "null" : value.toString();
    +			}
    +		};
    +		TableName tableName = TableName.valueOf("test");
    +		// no split keys
    +		byte[][] famNames = new byte[1][];
    +		famNames[0] = F_1;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19992l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993l));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		HBaseTableSchema schema = new HBaseTableSchema();
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_3), Long.class);
    +		// fetch row2 from the table till the end
    +		BatchTableSource hbaseTable = new HBaseTableSource(getConf(), tableName.getNameAsString(), schema);
    +		tableEnv.registerTableSource("test", hbaseTable);
    +		Table result = tableEnv
    +			.sql("SELECT test.f1.q1, test.f1.q2, test.f1.q3 FROM test");
    +		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    +		List<Row> results = resultSet.collect();
    +
    +		String expected = "100,strvalue,19991\n" +
    +			"101,strvalue1,19992\n" +
    +			"102,strvalue2,19993\n";
    +		compareResult(results, expected, false, true);
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithTwoColumnFamily() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test1");
    +		// no split keys
    +		byte[][] famNames = new byte[2][];
    +		famNames[0] = F_1;
    +		famNames[1] = F_2;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_2, Q_1, Bytes.toBytes(201));
    +		//2nd qual is String
    +		put.addColumn(F_2, Q_2, Bytes.toBytes("newvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_2, Q_3, Bytes.toBytes(29992l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993l));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		HBaseTableSchema schema = new HBaseTableSchema();
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_3), Long.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_3), Long.class);
    +		// fetch row2 from the table till the end
    +		BatchTableSource hbaseTable = new HBaseTableSource(getConf(), tableName.getNameAsString(), schema);
    +		tableEnv.registerTableSource("test1", hbaseTable);
    +		Table result = tableEnv
    +			.sql("SELECT test1.f1.q1, test1.f1.q2, test1.f1.q3, test1.f2.q1, test1.f2.q2, test1.f2.q3 FROM test1");
    +		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    +		List<Row> results = resultSet.collect();
    +
    +		String expected = "100,strvalue,19991,null,null,null\n" +
    +			"null,null,null,201,newvalue1,29992\n" +
    +			"102,strvalue2,19993,null,null,null\n";
    +		compareResult(results, expected, false, false);
    +	}
    +
    +
    +	static <T> void compareResult(List<T> result, String expected, boolean asTuples, boolean sort) {
    --- End diff --
    
    I think it's fine to add a test scope dependency. The `flink-test-utils_2.10` is used in many modules including `flink-connector-kafka`s.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Just a general question
    `def projectFields(fields: Array[Int]): ProjectableTableSource[T]`
    
    Is it mandatory to only have int[] here. Can we have String[] that allows to specify specific names? May be even more generic could be to have a ColumnarTableProjectableSource which allows to specify family to column mapping some way? Ultimately its the table source that is going to do the mapping and create the projected table source. So that should be fine?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Try to run in flink folder (I ran it onto ubuntu 16.04)
    ```bash
    export JAVA_HOME=/path/to/java-7-oracle/jre
    mvn -B -Dhadoop.version=2.3.0 \
    test -Dtest=org.apache.flink.addons.hbase.example.HBaseTableSourceITCase \
    -DfailIfNoTests=false
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97903708
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,248 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSchema;
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +	public static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	public static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	public static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	public static final byte[] F_1 = Bytes.toBytes("f1");
    +	public static final byte[] F_2 = Bytes.toBytes("f2");
    +	public static final byte[] Q_1 = Bytes.toBytes("q1");
    +	public static final byte[] Q_2 = Bytes.toBytes("q2");
    +	public static final byte[] Q_3 = Bytes.toBytes("q3");
    --- End diff --
    
    why public?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97031401
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
    +import org.apache.flink.api.common.io.RichInputFormat;
    +import org.apache.flink.api.common.io.statistics.BaseStatistics;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.io.InputSplitAssigner;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.client.Table;
    +import org.apache.hadoop.hbase.client.ClusterConnection;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.ResultScanner;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.HRegionLocator;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends RichInputFormat<Row, TableInputSplit> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private TypeInformation[] fieldTypeInfos;
    +	private String[] fieldNames;
    +	private transient Table table;
    +	private transient Scan scan;
    +	private transient Connection conn;
    +	private ResultScanner resultScanner = null;
    +
    +	private byte[] lastRow;
    +	private int scannedRows;
    +	private boolean endReached = false;
    +	private org.apache.hadoop.conf.Configuration conf;
    +	private static final String COLON = ":";
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, String[] fieldNames, TypeInformation[] fieldTypeInfos) {
    +		this.conf = conf;
    +		this.tableName = tableName;
    +		this.fieldNames = fieldNames;
    +		this.fieldTypeInfos = fieldTypeInfos;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = createScanner();
    +		}
    +	}
    +
    +	private Scan createScanner() {
    +		Scan scan = new Scan();
    +		for(String field : fieldNames) {
    +			// select only the fields in the 'selectedFields'
    +			String[] famCol = field.split(COLON);
    +			scan.addColumn(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +		}
    +		return scan;
    +	}
    +
    +	private void connectToTable() {
    +		//use files found in the classpath
    +		if(this.conf == null) {
    +			this.conf = HBaseConfiguration.create();
    +		}
    +		try {
    +			conn = ConnectionFactory.createConnection(this.conf);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while creating connection to hbase cluster", ioe);
    +			return;
    +		}
    +		try {
    +			table = conn.getTable(TableName.valueOf(tableName));
    +		} catch(TableNotFoundException tnfe) {
    +			LOG.error("The table " + tableName + " not found ", tnfe);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while connecting to the table "+tableName+ " ", ioe);
    +		}
    +	}
    +
    +	@Override
    +	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
    +		return null;
    +	}
    +
    +	@Override
    +	public TableInputSplit[] createInputSplits(final int minNumSplits) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +
    +		//Gets the starting and ending row keys for every region in the currently open table
    +		HRegionLocator regionLocator = new HRegionLocator(table.getName(), (ClusterConnection) conn);
    +		final Pair<byte[][], byte[][]> keys = regionLocator.getStartEndKeys();
    +		if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
    +			throw new IOException("Expecting at least one region.");
    +		}
    +		final byte[] startRow = scan.getStartRow();
    +		final byte[] stopRow = scan.getStopRow();
    +		final boolean scanWithNoLowerBound = startRow.length == 0;
    +		final boolean scanWithNoUpperBound = stopRow.length == 0;
    +
    +		final List<TableInputSplit> splits = new ArrayList<TableInputSplit>(minNumSplits);
    +		for (int i = 0; i < keys.getFirst().length; i++) {
    +			final byte[] startKey = keys.getFirst()[i];
    +			final byte[] endKey = keys.getSecond()[i];
    +			final String regionLocation = regionLocator.getRegionLocation(startKey, false).getHostnamePort();
    +			//Test if the given region is to be included in the InputSplit while splitting the regions of a table
    +			if (!includeRegionInSplit(startKey, endKey)) {
    +				continue;
    +			}
    +			//Finds the region on which the given row is being served
    +			final String[] hosts = new String[]{regionLocation};
    +
    +			// determine if regions contains keys used by the scan
    +			boolean isLastRegion = endKey.length == 0;
    +			if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0) &&
    +				(scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) {
    +
    +				final byte[] splitStart = scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0 ? startKey : startRow;
    +				final byte[] splitStop = (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0)
    +					&& !isLastRegion ? endKey : stopRow;
    +				int id = splits.size();
    +				final TableInputSplit split = new TableInputSplit(id, hosts, table.getName().getName(), splitStart, splitStop);
    +				splits.add(split);
    +			}
    +		}
    +		LOG.info("Created " + splits.size() + " splits");
    +		for (TableInputSplit split : splits) {
    +			logSplitInfo("created", split);
    +		}
    +		return splits.toArray(new TableInputSplit[0]);
    +	}
    +
    +	protected boolean includeRegionInSplit(final byte[] startKey, final byte[] endKey) {
    +		return true;
    +	}
    +
    +	@Override
    +	public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
    +		return new LocatableInputSplitAssigner(inputSplits);
    +	}
    +
    +	@Override
    +	public void open(TableInputSplit split) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +		if (split == null) {
    +			throw new IOException("Input split is null!");
    +		}
    +
    +		logSplitInfo("opening", split);
    +		// set the start row and stop row from the splits
    +		scan.setStartRow(split.getStartRow());
    +		lastRow = split.getEndRow();
    +		scan.setStopRow(lastRow);
    +
    +		resultScanner = table.getScanner(scan);
    +		endReached = false;
    +		scannedRows = 0;
    +	}
    +
    +	private void logSplitInfo(String action, TableInputSplit split) {
    +		int splitId = split.getSplitNumber();
    +		String splitStart = Bytes.toString(split.getStartRow());
    +		String splitEnd = Bytes.toString(split.getEndRow());
    +		String splitStartKey = splitStart.isEmpty() ? "-" : splitStart;
    +		String splitStopKey = splitEnd.isEmpty() ? "-" : splitEnd;
    +		String[] hostnames = split.getHostnames();
    +		LOG.info("{} split (this={})[{}|{}|{}|{}]", action, this, splitId, hostnames, splitStartKey, splitStopKey);
    +	}
    +
    +	@Override
    +	public boolean reachedEnd() throws IOException {
    +		return endReached;
    +	}
    +
    +	@Override
    +	public Row nextRecord(Row reuse) throws IOException {
    +		if (resultScanner == null) {
    +			throw new IOException("No table result scanner provided!");
    +		}
    +		try {
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		} catch (Exception e) {
    +			resultScanner.close();
    +			//workaround for timeout on scan
    +			LOG.warn("Error after scan of " + scannedRows + " rows. Retry with a new scanner...", e);
    +			scan.setStartRow(lastRow);
    +			resultScanner = table.getScanner(scan);
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		}
    +		endReached = true;
    +		return null;
    +	}
    +
    +	private Row mapResultToRow(Result res) {
    +		Object[] values = new Object[fieldNames.length];
    +		int i = 0;
    +		for(String field : fieldNames) {
    +			String[] famCol = field.split(COLON);
    +			byte[] value = res.getValue(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +			TypeInformation typeInfo = fieldTypeInfos[i];
    +			if(typeInfo.isBasicType()) {
    +				if(typeInfo.getTypeClass() ==  Integer.class) {
    +					values[i] = Bytes.toInt(value);
    +				} else if(typeInfo.getTypeClass() == Short.class) {
    +					values[i] = Bytes.toShort(value);
    +				} else if(typeInfo.getTypeClass() == Float.class) {
    +					values[i] = Bytes.toFloat(value);
    +				} else if(typeInfo.getTypeClass() == Long.class) {
    +					values[i] = Bytes.toLong(value);
    +				} else if(typeInfo.getTypeClass() == String.class) {
    +					values[i] = Bytes.toString(value);
    +				} else if(typeInfo.getTypeClass() == Byte.class) {
    +					values[i] = value[0];
    +				} else if(typeInfo.getTypeClass() == Boolean.class) {
    +					values[i] = Bytes.toBoolean(value);
    +				} else if(typeInfo.getTypeClass() == Double.class) {
    +					values[i] = Bytes.toDouble(value);
    +				} else if(typeInfo.getTypeClass() == BigInteger.class) {
    +					values[i] = new BigInteger(value);
    +				} else if(typeInfo.getTypeClass() == BigDecimal.class) {
    +					values[i] = Bytes.toBigDecimal(value);
    +				} else if(typeInfo.getTypeClass() == Date.class) {
    +					values[i] = new Date(Bytes.toLong(value));
    +				}
    +			} else {
    +				// TODO for other types??
    --- End diff --
    
    I think it's fine to add start/endKey into HBaseTableSchema.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97934510
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,248 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSchema;
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +	public static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	public static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	public static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	public static final byte[] F_1 = Bytes.toBytes("f1");
    +	public static final byte[] F_2 = Bytes.toBytes("f2");
    +	public static final byte[] Q_1 = Bytes.toBytes("q1");
    +	public static final byte[] Q_2 = Bytes.toBytes("q2");
    +	public static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@BeforeClass
    +	public static void activateHBaseCluster(){
    +		registerHBaseMiniClusterInClasspath();
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithSingleColumnFamily() throws Exception {
    +		// create a table with single region
    +		MapFunction<Row, String> mapFunction = new MapFunction<Row, String>() {
    +
    +			@Override
    +			public String map(Row value) throws Exception {
    +				return value == null ? "null" : value.toString();
    +			}
    +		};
    +		TableName tableName = TableName.valueOf("test");
    +		// no split keys
    +		byte[][] famNames = new byte[1][];
    +		famNames[0] = F_1;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19992l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993l));
    --- End diff --
    
    ok


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97956419
  
    --- Diff: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java ---
    @@ -464,7 +464,7 @@ protected static File asFile(String path) {
     		compareResult(result, expected, asTuples, false);
     	}
     
    -	private static <T> void compareResult(List<T> result, String expected, boolean asTuples, boolean sort) {
    +	public static <T> void compareResult(List<T> result, String expected, boolean asTuples, boolean sort) {
    --- End diff --
    
    Can you revert this ? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98486712
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,140 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.sql.Time;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.sql.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    +		new HashMap<>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class, Time.class, byte[].class
    --- End diff --
    
    Ok. I can do. I have not used much of these Immutblecollection. Infact in our project we removed the dependency on this. Hence went on with  a default way. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97055300
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java ---
    @@ -0,0 +1,75 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +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;
    +
    +/**
    + * 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> {
    +
    +	private Configuration conf;
    +	private String tableName;
    +	private byte[] rowKey;
    +	private String[] colNames;
    +	private TypeInformation<?>[] colTypes;
    +
    +	public HBaseTableSource(Configuration conf, String tableName, byte[] rowKey, String[] colNames,
    +							TypeInformation<?>[] colTypes) {
    +		this.conf = conf;
    +		this.tableName = Preconditions.checkNotNull(tableName, "Table  name");
    +		this.rowKey = Preconditions.checkNotNull(rowKey, "Rowkey");
    --- End diff --
    
    `new RowTypeInfo(
    	new TypeInformation[]{
    		new RowTypeInfo(
    			new TypeInformation[]{
    				BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO},
    			new String[]{"name", "age"})
    	},
    	new String[]{"person"}
    );`
    I did the above change for the HBaseTableSource's return type and then made a simple query
    `		tableEnv.registerTableSource("test", hbaseTable);
    		Table result = tableEnv
    			.sql("SELECT f1.q1, f1.q2, f1.q3 FROM test");`
    It throws sql validation error and considers f1 as table name and says tableName 'f1' not found.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    I think its fine to change the JVM settings for the hbase module. Other modules such as elasticsearch are also doing that.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98805953
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,196 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSchema;
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.test.util.TestBaseUtils;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +	private static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	private static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	private static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	private static final byte[] F_1 = Bytes.toBytes("f1");
    +	private static final byte[] F_2 = Bytes.toBytes("f2");
    +	private static final byte[] Q_1 = Bytes.toBytes("q1");
    +	private static final byte[] Q_2 = Bytes.toBytes("q2");
    +	private static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@BeforeClass
    +	public static void activateHBaseCluster(){
    +		registerHBaseMiniClusterInClasspath();
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithSingleColumnFamily() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test");
    +		// no split keys
    +		byte[][] famNames = new byte[1][];
    +		famNames[0] = F_1;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19992L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993L));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		HBaseTableSchema schema = new HBaseTableSchema();
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_3), Long.class);
    +		// fetch row2 from the table till the end
    +		BatchTableSource hbaseTable = new HBaseTableSource(getConf(), tableName.getNameAsString(), schema);
    +		tableEnv.registerTableSource("test", hbaseTable);
    +		Table result = tableEnv
    +			.sql("SELECT test.f1.q1, test.f1.q2, test.f1.q3 FROM test");
    +		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    +		List<Row> results = resultSet.collect();
    +
    +		String expected = "100,strvalue,19991\n" +
    +			"101,strvalue1,19992\n" +
    +			"102,strvalue2,19993\n";
    +		TestBaseUtils.compareResultAsText(results, expected);
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithTwoColumnFamily() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test1");
    +		// no split keys
    +		byte[][] famNames = new byte[2][];
    +		famNames[0] = F_1;
    +		famNames[1] = F_2;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_2, Q_1, Bytes.toBytes(201));
    +		//2nd qual is String
    +		put.addColumn(F_2, Q_2, Bytes.toBytes("newvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_2, Q_3, Bytes.toBytes(29992L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993L));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    --- End diff --
    
    same comments as in test above


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    > Results :
    
    Tests run: 2, Failures: 0, Errors: 0, Skipped: 0
    
    This is what I get as test result.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98828506
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends TableInputFormat<Row> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private transient Connection conn;
    +	private transient org.apache.hadoop.conf.Configuration conf;
    +	private HBaseTableSchema schema;
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, HBaseTableSchema schema) {
    +		this.tableName = tableName;
    +		this.conf = conf;
    +		this.schema = schema;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = getScanner();
    +		}
    +	}
    +
    +	@Override
    +	protected Scan getScanner() {
    +		Scan scan = new Scan();
    +		for(String family : schema.getFamilyNames()) {
    --- End diff --
    
    Actually in hbse every thing is lexographically sorted (with byte[]). But here since we represent using String the map that we use in HBaseTableSchema should be sorting this in the natural order. 
    But the results that we retrieve will always be lexographically sorted and we try to retrieve the specific cell from the result using this
    
    >  res.getValue(Bytes.toBytes(family), Bytes.toBytes(qualifier));
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97709518
  
    --- 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 --
    
    I will look into this.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    > For now I'd suggest to keep the scope of the PR as it is right now. A bit more Java documentation on HBaseTableSource to explain how it is used would be great.
    We can implement the NestedFieldsProjectableTableSource and the changes to HBaseTableSource in a follow up issue.
    
    +1 for this. I can add some more javadoc to it. BTW am trying to checkout these Projections and using the ProjectabletableSource. Will be back on it. Thanks to every one for all the comments and feedback.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97015321
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java ---
    @@ -0,0 +1,75 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +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;
    +
    +/**
    + * 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> {
    +
    +	private Configuration conf;
    +	private String tableName;
    +	private byte[] rowKey;
    +	private String[] colNames;
    +	private TypeInformation<?>[] colTypes;
    +
    +	public HBaseTableSource(Configuration conf, String tableName, byte[] rowKey, String[] colNames,
    +							TypeInformation<?>[] colTypes) {
    +		this.conf = conf;
    +		this.tableName = Preconditions.checkNotNull(tableName, "Table  name");
    +		this.rowKey = Preconditions.checkNotNull(rowKey, "Rowkey");
    --- End diff --
    
    Yes. That is true but do we always want full table scan? Actually in HBase it is better we specify start and end key. So how do we specify that? I have not used this rowKey now but I thought it is better to be used?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @ramkrish86 what is your environment?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97730820
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair>> familyMap =
    +		new HashMap<String, List<Pair>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class
    +	};
    +	private static byte[] EMPTY_BYTE_ARRAY = new byte[0];
    +	public void addColumns(String family, String qualifier, TypeInformation<?> type) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(family, "qualifier name");
    +		Preconditions.checkNotNull(type, "type name");
    +		List<Pair> list = this.familyMap.get(family);
    +		if (list == null) {
    +			list = new ArrayList<Pair>();
    +		}
    +		boolean found = false;
    +		for(Class classType : CLASS_TYPES) {
    +			if(classType == type.getTypeClass()) {
    +				found = true;
    +				break;
    +			}
    +		}
    +		if(!found) {
    +			// by default it will be byte[] type only
    +			type = BasicArrayTypeInfo.BYTE_ARRAY_TYPE_INFO;
    +		}
    +		list.add(new Pair(qualifier, type));
    +		familyMap.put(family, list);
    +	}
    +
    +	public Map<String, List<Pair>> getFamilyMap() {
    +		return this.familyMap;
    +	}
    +
    +	public Object deserialize(byte[] value, TypeInformation<?> typeInfo) {
    +		if (typeInfo.isBasicType()) {
    +			if (typeInfo.getTypeClass() == Integer.class) {
    +				return Bytes.toInt(value);
    +			} else if (typeInfo.getTypeClass() == Short.class) {
    +				return Bytes.toShort(value);
    +			} else if (typeInfo.getTypeClass() == Float.class) {
    +				return Bytes.toFloat(value);
    +			} else if (typeInfo.getTypeClass() == Long.class) {
    +				return Bytes.toLong(value);
    +			} else if (typeInfo.getTypeClass() == String.class) {
    +				return Bytes.toString(value);
    +			} else if (typeInfo.getTypeClass() == Byte.class) {
    +				return value[0];
    +			} else if (typeInfo.getTypeClass() == Boolean.class) {
    +				return Bytes.toBoolean(value);
    +			} else if (typeInfo.getTypeClass() == Double.class) {
    +				return Bytes.toDouble(value);
    +			} else if (typeInfo.getTypeClass() == BigInteger.class) {
    +				return new BigInteger(value);
    +			} else if (typeInfo.getTypeClass() == BigDecimal.class) {
    +				return Bytes.toBigDecimal(value);
    +			} else if (typeInfo.getTypeClass() == Date.class) {
    +				return new Date(Bytes.toLong(value));
    +			}
    +		}
    +		return value;
    +	}
    +
    +	public Object deserializeNull(TypeInformation<?> typeInfo) {
    +		// TODO : this may need better handling.
    +		if(typeInfo.getTypeClass() ==  Integer.class) {
    --- End diff --
    
    I don't think so. `CodeGenUtils#primitiveDefaultValue` is used to give a default value to primitives. Flink code generation use primitives to improve performance. But the field of `Row` is not primitive which can be `null`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97904966
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,248 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSchema;
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +	public static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	public static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	public static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	public static final byte[] F_1 = Bytes.toBytes("f1");
    +	public static final byte[] F_2 = Bytes.toBytes("f2");
    +	public static final byte[] Q_1 = Bytes.toBytes("q1");
    +	public static final byte[] Q_2 = Bytes.toBytes("q2");
    +	public static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@BeforeClass
    +	public static void activateHBaseCluster(){
    +		registerHBaseMiniClusterInClasspath();
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithSingleColumnFamily() throws Exception {
    +		// create a table with single region
    +		MapFunction<Row, String> mapFunction = new MapFunction<Row, String>() {
    +
    +			@Override
    +			public String map(Row value) throws Exception {
    +				return value == null ? "null" : value.toString();
    +			}
    +		};
    +		TableName tableName = TableName.valueOf("test");
    +		// no split keys
    +		byte[][] famNames = new byte[1][];
    +		famNames[0] = F_1;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19992l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993l));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		HBaseTableSchema schema = new HBaseTableSchema();
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_3), Long.class);
    +		// fetch row2 from the table till the end
    +		BatchTableSource hbaseTable = new HBaseTableSource(getConf(), tableName.getNameAsString(), schema);
    +		tableEnv.registerTableSource("test", hbaseTable);
    +		Table result = tableEnv
    +			.sql("SELECT test.f1.q1, test.f1.q2, test.f1.q3 FROM test");
    +		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    +		List<Row> results = resultSet.collect();
    +
    +		String expected = "100,strvalue,19991\n" +
    +			"101,strvalue1,19992\n" +
    +			"102,strvalue2,19993\n";
    +		compareResult(results, expected, false, true);
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithTwoColumnFamily() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test1");
    +		// no split keys
    +		byte[][] famNames = new byte[2][];
    +		famNames[0] = F_1;
    +		famNames[1] = F_2;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_2, Q_1, Bytes.toBytes(201));
    +		//2nd qual is String
    +		put.addColumn(F_2, Q_2, Bytes.toBytes("newvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_2, Q_3, Bytes.toBytes(29992l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993l));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		HBaseTableSchema schema = new HBaseTableSchema();
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_3), Long.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_3), Long.class);
    +		// fetch row2 from the table till the end
    +		BatchTableSource hbaseTable = new HBaseTableSource(getConf(), tableName.getNameAsString(), schema);
    +		tableEnv.registerTableSource("test1", hbaseTable);
    +		Table result = tableEnv
    +			.sql("SELECT test1.f1.q1, test1.f1.q2, test1.f1.q3, test1.f2.q1, test1.f2.q2, test1.f2.q3 FROM test1");
    +		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    +		List<Row> results = resultSet.collect();
    +
    +		String expected = "100,strvalue,19991,null,null,null\n" +
    +			"null,null,null,201,newvalue1,29992\n" +
    +			"102,strvalue2,19993,null,null,null\n";
    +		compareResult(results, expected, false, false);
    +	}
    +
    +
    +	static <T> void compareResult(List<T> result, String expected, boolean asTuples, boolean sort) {
    --- End diff --
    
    big copypaste, i think there is a better way - decouple `TestBaseUtils` in two classes - cluster starter and result checker. I will start conversation in dev list


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Hi @ramkrish86, @tonycox could you reproduce the OOME on a local setup? 
    We might want to compare the new HBase InputFormat and the existing `TableInputFormat` which does not fail in tests.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97898531
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    +		new HashMap<String, List<Pair<String, TypeInformation<?>>>>();
    --- End diff --
    
    could you just put `<>` here instead of <String, List<Pair<String, TypeInformation<?>>>>?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Good news is that with the help of this Composite RowType and modifying my code accordingly and debugging things I could get the basic thing to work. Now I will work on stitching things together and submitting a PR with updated changes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Thanks for the update @ramkrish86 and your patience!
    I skimmed over the PR and it looks good.
    I'll do a final pass early next week before merging it.
    
    Thanks, Fabian


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    It's not only Java 7. For Hadoop 2.4.1 and Java 7, the builds pass. So it seems to be a combination of both parameters. Anyway, I think we can configure the JVM for the tests. 
    
    It would be good to check if @rmetzger has any objections. 
    He knows Maven and testing infrastructure much better than I do.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97927186
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,248 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSchema;
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +	public static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	public static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	public static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	public static final byte[] F_1 = Bytes.toBytes("f1");
    +	public static final byte[] F_2 = Bytes.toBytes("f2");
    +	public static final byte[] Q_1 = Bytes.toBytes("q1");
    +	public static final byte[] Q_2 = Bytes.toBytes("q2");
    +	public static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@BeforeClass
    +	public static void activateHBaseCluster(){
    +		registerHBaseMiniClusterInClasspath();
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithSingleColumnFamily() throws Exception {
    +		// create a table with single region
    +		MapFunction<Row, String> mapFunction = new MapFunction<Row, String>() {
    +
    +			@Override
    +			public String map(Row value) throws Exception {
    +				return value == null ? "null" : value.toString();
    +			}
    +		};
    +		TableName tableName = TableName.valueOf("test");
    +		// no split keys
    +		byte[][] famNames = new byte[1][];
    +		famNames[0] = F_1;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19992l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993l));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		HBaseTableSchema schema = new HBaseTableSchema();
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_3), Long.class);
    +		// fetch row2 from the table till the end
    +		BatchTableSource hbaseTable = new HBaseTableSource(getConf(), tableName.getNameAsString(), schema);
    +		tableEnv.registerTableSource("test", hbaseTable);
    +		Table result = tableEnv
    +			.sql("SELECT test.f1.q1, test.f1.q2, test.f1.q3 FROM test");
    +		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    +		List<Row> results = resultSet.collect();
    +
    +		String expected = "100,strvalue,19991\n" +
    +			"101,strvalue1,19992\n" +
    +			"102,strvalue2,19993\n";
    +		compareResult(results, expected, false, true);
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithTwoColumnFamily() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test1");
    +		// no split keys
    +		byte[][] famNames = new byte[2][];
    +		famNames[0] = F_1;
    +		famNames[1] = F_2;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_2, Q_1, Bytes.toBytes(201));
    +		//2nd qual is String
    +		put.addColumn(F_2, Q_2, Bytes.toBytes("newvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_2, Q_3, Bytes.toBytes(29992l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993l));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		HBaseTableSchema schema = new HBaseTableSchema();
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_3), Long.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_3), Long.class);
    +		// fetch row2 from the table till the end
    +		BatchTableSource hbaseTable = new HBaseTableSource(getConf(), tableName.getNameAsString(), schema);
    +		tableEnv.registerTableSource("test1", hbaseTable);
    +		Table result = tableEnv
    +			.sql("SELECT test1.f1.q1, test1.f1.q2, test1.f1.q3, test1.f2.q1, test1.f2.q2, test1.f2.q3 FROM test1");
    +		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    +		List<Row> results = resultSet.collect();
    +
    +		String expected = "100,strvalue,19991,null,null,null\n" +
    +			"null,null,null,201,newvalue1,29992\n" +
    +			"102,strvalue2,19993,null,null,null\n";
    +		compareResult(results, expected, false, false);
    +	}
    +
    +
    +	static <T> void compareResult(List<T> result, String expected, boolean asTuples, boolean sort) {
    --- End diff --
    
    @tonycox is right. This is a copypaste. But the methods in `TestBaseUtils` are static, we do not need to decouple it. We can use it directly like following:
    
    ```java
    TestBaseUtils.compareResultAsText(results, expected);
    ```
    
    Of course, we should add the `flink-test-utils` dependency first.
    
    ```
                   <dependency>
    			<groupId>org.apache.flink</groupId>
    			<artifactId>flink-test-utils_2.10</artifactId>
    			<version>${project.version}</version>
    			<scope>test</scope>
    		</dependency>
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97709565
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends TableInputFormat<Row> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private transient Connection conn;
    +	private transient org.apache.hadoop.conf.Configuration conf;
    +	private HBaseTableSchema schema;
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, HBaseTableSchema schema) {
    +		this.tableName = tableName;
    +		this.conf = conf;
    +		this.schema = schema;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = getScanner();
    +		}
    +	}
    +
    +	@Override
    +	protected Scan getScanner() {
    +		// TODO : Pass 'rowkey'. For this we need FilterableTableSource
    +		Scan scan = new Scan();
    +		Map<String, List<Pair>> familyMap = schema.getFamilyMap();
    +		for(String family : familyMap.keySet()) {
    +			// select only the fields in the 'selectedFields'
    +			List<Pair> colDetails = familyMap.get(family);
    +			for(Pair<String, TypeInformation<?>> pair : colDetails) {
    +				scan.addColumn(Bytes.toBytes(family), Bytes.toBytes(pair.getFirst()));
    +			}
    +		}
    +		return scan;
    +	}
    +
    +	@Override
    +	public String getTableName() {
    +		return tableName;
    +	}
    +
    +	@Override
    +	protected Row mapResultToTuple(Result res) {
    +		List<Object> values = new ArrayList<Object>();
    +		int i = 0;
    +		Map<String, List<Pair>> familyMap = schema.getFamilyMap();
    +		Row[] rows = new Row[familyMap.size()];
    +		for(String family : familyMap.keySet()) {
    +			List<Pair> colDetails = familyMap.get(family);
    +			for(Pair<String, TypeInformation<?>> pair : colDetails) {
    +				byte[] value = res.getValue(Bytes.toBytes(family), Bytes.toBytes(pair.getFirst()));
    +				if(value != null) {
    +					values.add(schema.deserialize(value, pair.getSecond()));
    +				} else {
    +					values.add(schema.deserializeNull(pair.getSecond()));
    --- End diff --
    
    Ya. I dint know this. Your question answers my doubt and the shaky method that I added can be removed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r96823925
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
    +import org.apache.flink.api.common.io.RichInputFormat;
    +import org.apache.flink.api.common.io.statistics.BaseStatistics;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.io.InputSplitAssigner;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.client.Table;
    +import org.apache.hadoop.hbase.client.ClusterConnection;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.ResultScanner;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.HRegionLocator;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends RichInputFormat<Row, TableInputSplit> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private TypeInformation[] fieldTypeInfos;
    +	private String[] fieldNames;
    +	private transient Table table;
    +	private transient Scan scan;
    +	private transient Connection conn;
    +	private ResultScanner resultScanner = null;
    +
    +	private byte[] lastRow;
    +	private int scannedRows;
    +	private boolean endReached = false;
    +	private org.apache.hadoop.conf.Configuration conf;
    +	private static final String COLON = ":";
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, String[] fieldNames, TypeInformation[] fieldTypeInfos) {
    +		this.conf = conf;
    +		this.tableName = tableName;
    +		this.fieldNames = fieldNames;
    +		this.fieldTypeInfos = fieldTypeInfos;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = createScanner();
    +		}
    +	}
    +
    +	private Scan createScanner() {
    +		Scan scan = new Scan();
    +		for(String field : fieldNames) {
    +			// select only the fields in the 'selectedFields'
    +			String[] famCol = field.split(COLON);
    +			scan.addColumn(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +		}
    +		return scan;
    +	}
    +
    +	private void connectToTable() {
    +		//use files found in the classpath
    +		if(this.conf == null) {
    +			this.conf = HBaseConfiguration.create();
    +		}
    +		try {
    +			conn = ConnectionFactory.createConnection(this.conf);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while creating connection to hbase cluster", ioe);
    +			return;
    +		}
    +		try {
    +			table = conn.getTable(TableName.valueOf(tableName));
    +		} catch(TableNotFoundException tnfe) {
    +			LOG.error("The table " + tableName + " not found ", tnfe);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while connecting to the table "+tableName+ " ", ioe);
    +		}
    +	}
    +
    +	@Override
    +	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
    +		return null;
    +	}
    +
    +	@Override
    +	public TableInputSplit[] createInputSplits(final int minNumSplits) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +
    +		//Gets the starting and ending row keys for every region in the currently open table
    +		HRegionLocator regionLocator = new HRegionLocator(table.getName(), (ClusterConnection) conn);
    +		final Pair<byte[][], byte[][]> keys = regionLocator.getStartEndKeys();
    +		if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
    +			throw new IOException("Expecting at least one region.");
    +		}
    +		final byte[] startRow = scan.getStartRow();
    +		final byte[] stopRow = scan.getStopRow();
    +		final boolean scanWithNoLowerBound = startRow.length == 0;
    +		final boolean scanWithNoUpperBound = stopRow.length == 0;
    +
    +		final List<TableInputSplit> splits = new ArrayList<TableInputSplit>(minNumSplits);
    +		for (int i = 0; i < keys.getFirst().length; i++) {
    +			final byte[] startKey = keys.getFirst()[i];
    +			final byte[] endKey = keys.getSecond()[i];
    +			final String regionLocation = regionLocator.getRegionLocation(startKey, false).getHostnamePort();
    +			//Test if the given region is to be included in the InputSplit while splitting the regions of a table
    +			if (!includeRegionInSplit(startKey, endKey)) {
    +				continue;
    +			}
    +			//Finds the region on which the given row is being served
    +			final String[] hosts = new String[]{regionLocation};
    +
    +			// determine if regions contains keys used by the scan
    +			boolean isLastRegion = endKey.length == 0;
    +			if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0) &&
    +				(scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) {
    +
    +				final byte[] splitStart = scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0 ? startKey : startRow;
    +				final byte[] splitStop = (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0)
    +					&& !isLastRegion ? endKey : stopRow;
    +				int id = splits.size();
    +				final TableInputSplit split = new TableInputSplit(id, hosts, table.getName().getName(), splitStart, splitStop);
    +				splits.add(split);
    +			}
    +		}
    +		LOG.info("Created " + splits.size() + " splits");
    +		for (TableInputSplit split : splits) {
    +			logSplitInfo("created", split);
    +		}
    +		return splits.toArray(new TableInputSplit[0]);
    +	}
    +
    +	protected boolean includeRegionInSplit(final byte[] startKey, final byte[] endKey) {
    +		return true;
    +	}
    +
    +	@Override
    +	public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
    +		return new LocatableInputSplitAssigner(inputSplits);
    +	}
    +
    +	@Override
    +	public void open(TableInputSplit split) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +		if (split == null) {
    +			throw new IOException("Input split is null!");
    +		}
    +
    +		logSplitInfo("opening", split);
    +		// set the start row and stop row from the splits
    +		scan.setStartRow(split.getStartRow());
    +		lastRow = split.getEndRow();
    +		scan.setStopRow(lastRow);
    +
    +		resultScanner = table.getScanner(scan);
    +		endReached = false;
    +		scannedRows = 0;
    +	}
    +
    +	private void logSplitInfo(String action, TableInputSplit split) {
    +		int splitId = split.getSplitNumber();
    +		String splitStart = Bytes.toString(split.getStartRow());
    +		String splitEnd = Bytes.toString(split.getEndRow());
    +		String splitStartKey = splitStart.isEmpty() ? "-" : splitStart;
    +		String splitStopKey = splitEnd.isEmpty() ? "-" : splitEnd;
    +		String[] hostnames = split.getHostnames();
    +		LOG.info("{} split (this={})[{}|{}|{}|{}]", action, this, splitId, hostnames, splitStartKey, splitStopKey);
    +	}
    +
    +	@Override
    +	public boolean reachedEnd() throws IOException {
    +		return endReached;
    +	}
    +
    +	@Override
    +	public Row nextRecord(Row reuse) throws IOException {
    +		if (resultScanner == null) {
    +			throw new IOException("No table result scanner provided!");
    +		}
    +		try {
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		} catch (Exception e) {
    +			resultScanner.close();
    +			//workaround for timeout on scan
    +			LOG.warn("Error after scan of " + scannedRows + " rows. Retry with a new scanner...", e);
    +			scan.setStartRow(lastRow);
    +			resultScanner = table.getScanner(scan);
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		}
    +		endReached = true;
    +		return null;
    +	}
    +
    +	private Row mapResultToRow(Result res) {
    +		Object[] values = new Object[fieldNames.length];
    +		int i = 0;
    +		for(String field : fieldNames) {
    +			String[] famCol = field.split(COLON);
    +			byte[] value = res.getValue(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +			TypeInformation typeInfo = fieldTypeInfos[i];
    +			if(typeInfo.isBasicType()) {
    +				if(typeInfo.getTypeClass() ==  Integer.class) {
    +					values[i] = Bytes.toInt(value);
    +				} else if(typeInfo.getTypeClass() == Short.class) {
    +					values[i] = Bytes.toShort(value);
    +				} else if(typeInfo.getTypeClass() == Float.class) {
    +					values[i] = Bytes.toFloat(value);
    +				} else if(typeInfo.getTypeClass() == Long.class) {
    +					values[i] = Bytes.toLong(value);
    +				} else if(typeInfo.getTypeClass() == String.class) {
    +					values[i] = Bytes.toString(value);
    +				} else if(typeInfo.getTypeClass() == Byte.class) {
    +					values[i] = value[0];
    +				} else if(typeInfo.getTypeClass() == Boolean.class) {
    +					values[i] = Bytes.toBoolean(value);
    +				} else if(typeInfo.getTypeClass() == Double.class) {
    +					values[i] = Bytes.toDouble(value);
    +				} else if(typeInfo.getTypeClass() == BigInteger.class) {
    +					values[i] = new BigInteger(value);
    +				} else if(typeInfo.getTypeClass() == BigDecimal.class) {
    +					values[i] = Bytes.toBigDecimal(value);
    +				} else if(typeInfo.getTypeClass() == Date.class) {
    +					values[i] = new Date(Bytes.toLong(value));
    +				}
    +			} else {
    +				// TODO for other types??
    --- End diff --
    
    No need to implement `DefinedFieldNames`. The [f0, f1, f2] is the default field names of RowTypeInfo, you should set the custom field names in RowTypeInfo. Using `RowTypeInfo(TypeInformation<?>[] types, String[] fieldNames)` constructor. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Sounds good ! Looking forward that !


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    `		Table result = tableEnv
    			.sql("SELECT test1.f1.q1, test1.f2.q2 FROM test1 where test1.f1.q1 < 103");`
    
    I just tried this query and it works with or without ProjectableTableSource.  So just wanted to know when does the projection come into place. I thought without Projection things of this sort may not work. 
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r96790136
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
    +import org.apache.flink.api.common.io.RichInputFormat;
    +import org.apache.flink.api.common.io.statistics.BaseStatistics;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.io.InputSplitAssigner;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.client.Table;
    +import org.apache.hadoop.hbase.client.ClusterConnection;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.ResultScanner;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.HRegionLocator;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends RichInputFormat<Row, TableInputSplit> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private TypeInformation[] fieldTypeInfos;
    +	private String[] fieldNames;
    +	private transient Table table;
    +	private transient Scan scan;
    +	private transient Connection conn;
    +	private ResultScanner resultScanner = null;
    +
    +	private byte[] lastRow;
    +	private int scannedRows;
    +	private boolean endReached = false;
    +	private org.apache.hadoop.conf.Configuration conf;
    +	private static final String COLON = ":";
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, String[] fieldNames, TypeInformation[] fieldTypeInfos) {
    +		this.conf = conf;
    +		this.tableName = tableName;
    +		this.fieldNames = fieldNames;
    +		this.fieldTypeInfos = fieldTypeInfos;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = createScanner();
    +		}
    +	}
    +
    +	private Scan createScanner() {
    +		Scan scan = new Scan();
    +		for(String field : fieldNames) {
    +			// select only the fields in the 'selectedFields'
    +			String[] famCol = field.split(COLON);
    +			scan.addColumn(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +		}
    +		return scan;
    +	}
    +
    +	private void connectToTable() {
    +		//use files found in the classpath
    +		if(this.conf == null) {
    +			this.conf = HBaseConfiguration.create();
    +		}
    +		try {
    +			conn = ConnectionFactory.createConnection(this.conf);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while creating connection to hbase cluster", ioe);
    +			return;
    +		}
    +		try {
    +			table = conn.getTable(TableName.valueOf(tableName));
    +		} catch(TableNotFoundException tnfe) {
    +			LOG.error("The table " + tableName + " not found ", tnfe);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while connecting to the table "+tableName+ " ", ioe);
    +		}
    +	}
    +
    +	@Override
    +	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
    +		return null;
    +	}
    +
    +	@Override
    +	public TableInputSplit[] createInputSplits(final int minNumSplits) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +
    +		//Gets the starting and ending row keys for every region in the currently open table
    +		HRegionLocator regionLocator = new HRegionLocator(table.getName(), (ClusterConnection) conn);
    +		final Pair<byte[][], byte[][]> keys = regionLocator.getStartEndKeys();
    +		if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
    +			throw new IOException("Expecting at least one region.");
    +		}
    +		final byte[] startRow = scan.getStartRow();
    +		final byte[] stopRow = scan.getStopRow();
    +		final boolean scanWithNoLowerBound = startRow.length == 0;
    +		final boolean scanWithNoUpperBound = stopRow.length == 0;
    +
    +		final List<TableInputSplit> splits = new ArrayList<TableInputSplit>(minNumSplits);
    +		for (int i = 0; i < keys.getFirst().length; i++) {
    +			final byte[] startKey = keys.getFirst()[i];
    +			final byte[] endKey = keys.getSecond()[i];
    +			final String regionLocation = regionLocator.getRegionLocation(startKey, false).getHostnamePort();
    +			//Test if the given region is to be included in the InputSplit while splitting the regions of a table
    +			if (!includeRegionInSplit(startKey, endKey)) {
    +				continue;
    +			}
    +			//Finds the region on which the given row is being served
    +			final String[] hosts = new String[]{regionLocation};
    +
    +			// determine if regions contains keys used by the scan
    +			boolean isLastRegion = endKey.length == 0;
    +			if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0) &&
    +				(scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) {
    +
    +				final byte[] splitStart = scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0 ? startKey : startRow;
    +				final byte[] splitStop = (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0)
    +					&& !isLastRegion ? endKey : stopRow;
    +				int id = splits.size();
    +				final TableInputSplit split = new TableInputSplit(id, hosts, table.getName().getName(), splitStart, splitStop);
    +				splits.add(split);
    +			}
    +		}
    +		LOG.info("Created " + splits.size() + " splits");
    +		for (TableInputSplit split : splits) {
    +			logSplitInfo("created", split);
    +		}
    +		return splits.toArray(new TableInputSplit[0]);
    +	}
    +
    +	protected boolean includeRegionInSplit(final byte[] startKey, final byte[] endKey) {
    +		return true;
    +	}
    +
    +	@Override
    +	public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
    +		return new LocatableInputSplitAssigner(inputSplits);
    +	}
    +
    +	@Override
    +	public void open(TableInputSplit split) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +		if (split == null) {
    +			throw new IOException("Input split is null!");
    +		}
    +
    +		logSplitInfo("opening", split);
    +		// set the start row and stop row from the splits
    +		scan.setStartRow(split.getStartRow());
    +		lastRow = split.getEndRow();
    +		scan.setStopRow(lastRow);
    +
    +		resultScanner = table.getScanner(scan);
    +		endReached = false;
    +		scannedRows = 0;
    +	}
    +
    +	private void logSplitInfo(String action, TableInputSplit split) {
    +		int splitId = split.getSplitNumber();
    +		String splitStart = Bytes.toString(split.getStartRow());
    +		String splitEnd = Bytes.toString(split.getEndRow());
    +		String splitStartKey = splitStart.isEmpty() ? "-" : splitStart;
    +		String splitStopKey = splitEnd.isEmpty() ? "-" : splitEnd;
    +		String[] hostnames = split.getHostnames();
    +		LOG.info("{} split (this={})[{}|{}|{}|{}]", action, this, splitId, hostnames, splitStartKey, splitStopKey);
    +	}
    +
    +	@Override
    +	public boolean reachedEnd() throws IOException {
    +		return endReached;
    +	}
    +
    +	@Override
    +	public Row nextRecord(Row reuse) throws IOException {
    +		if (resultScanner == null) {
    +			throw new IOException("No table result scanner provided!");
    +		}
    +		try {
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		} catch (Exception e) {
    +			resultScanner.close();
    +			//workaround for timeout on scan
    +			LOG.warn("Error after scan of " + scannedRows + " rows. Retry with a new scanner...", e);
    +			scan.setStartRow(lastRow);
    +			resultScanner = table.getScanner(scan);
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		}
    +		endReached = true;
    +		return null;
    +	}
    +
    +	private Row mapResultToRow(Result res) {
    +		Object[] values = new Object[fieldNames.length];
    +		int i = 0;
    +		for(String field : fieldNames) {
    +			String[] famCol = field.split(COLON);
    +			byte[] value = res.getValue(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +			TypeInformation typeInfo = fieldTypeInfos[i];
    +			if(typeInfo.isBasicType()) {
    +				if(typeInfo.getTypeClass() ==  Integer.class) {
    +					values[i] = Bytes.toInt(value);
    +				} else if(typeInfo.getTypeClass() == Short.class) {
    +					values[i] = Bytes.toShort(value);
    +				} else if(typeInfo.getTypeClass() == Float.class) {
    +					values[i] = Bytes.toFloat(value);
    +				} else if(typeInfo.getTypeClass() == Long.class) {
    +					values[i] = Bytes.toLong(value);
    +				} else if(typeInfo.getTypeClass() == String.class) {
    +					values[i] = Bytes.toString(value);
    +				} else if(typeInfo.getTypeClass() == Byte.class) {
    +					values[i] = value[0];
    +				} else if(typeInfo.getTypeClass() == Boolean.class) {
    +					values[i] = Bytes.toBoolean(value);
    +				} else if(typeInfo.getTypeClass() == Double.class) {
    +					values[i] = Bytes.toDouble(value);
    +				} else if(typeInfo.getTypeClass() == BigInteger.class) {
    +					values[i] = new BigInteger(value);
    +				} else if(typeInfo.getTypeClass() == BigDecimal.class) {
    +					values[i] = Bytes.toBigDecimal(value);
    +				} else if(typeInfo.getTypeClass() == Date.class) {
    +					values[i] = new Date(Bytes.toLong(value));
    +				}
    +			} else {
    +				// TODO for other types??
    --- End diff --
    
    I can see that in the TableSourceITCase that uses CsvInputFormat - the metadata is also passed as the input and the meta data is registered with the table. Now in cases like hbase - how should that be done? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98486522
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,140 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.sql.Time;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.sql.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    --- End diff --
    
    I just felt Map is not needed here. Anyway we have the Key as String. If you are so particular then I can change. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @tonycox 
    I have addressed all your latest comments including making HBaseTableSource a ProjectableTableSource. 
    @wuchong , @fhueske 
    Are you guys fine with the latest updates. If so we can try closing this PR and further discussions of adding StreamingtableSource and supporting WHERE clauses can be done in subsequent PRs.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97030686
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java ---
    @@ -0,0 +1,75 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +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;
    +
    +/**
    + * 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> {
    +
    +	private Configuration conf;
    +	private String tableName;
    +	private byte[] rowKey;
    +	private String[] colNames;
    +	private TypeInformation<?>[] colTypes;
    +
    +	public HBaseTableSource(Configuration conf, String tableName, byte[] rowKey, String[] colNames,
    +							TypeInformation<?>[] colTypes) {
    +		this.conf = conf;
    +		this.tableName = Preconditions.checkNotNull(tableName, "Table  name");
    +		this.rowKey = Preconditions.checkNotNull(rowKey, "Rowkey");
    --- End diff --
    
    Make sense. We can support the start and end rowkey.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    We need discuss that


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    For ProjectableTableSource I think I need some clarity. Because currently is based on int[] representing the fields. So am not sure how to map them in terms of qualifiers under a family.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97934446
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    +		new HashMap<String, List<Pair<String, TypeInformation<?>>>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class, byte[].class
    +	};
    +	/**
    +	 * Allows specifying the family and qualifier name along with the data type of the qualifier for an HBase table
    +	 *
    +	 * @param family    the family name
    +	 * @param qualifier the qualifier name
    +	 * @param clazz     the data type of the qualifier
    +	 */
    +	public void addColumn(String family, String qualifier, Class<?> clazz) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(family, "qualifier name");
    --- End diff --
    
    Good catch.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97696842
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair>> familyMap =
    +		new HashMap<String, List<Pair>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class
    +	};
    +	private static byte[] EMPTY_BYTE_ARRAY = new byte[0];
    +	public void addColumns(String family, String qualifier, TypeInformation<?> type) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(family, "qualifier name");
    +		Preconditions.checkNotNull(type, "type name");
    +		List<Pair> list = this.familyMap.get(family);
    +		if (list == null) {
    +			list = new ArrayList<Pair>();
    +		}
    +		boolean found = false;
    +		for(Class classType : CLASS_TYPES) {
    +			if(classType == type.getTypeClass()) {
    +				found = true;
    +				break;
    +			}
    +		}
    +		if(!found) {
    +			// by default it will be byte[] type only
    +			type = BasicArrayTypeInfo.BYTE_ARRAY_TYPE_INFO;
    --- End diff --
    
    I think we should throw an exception here to indicate users this class type is not supported, please use byte[].class instead. Otherwise, the user will think that this type can work, and select the value without deserialization, and get the unexpected result.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r96666157
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,117 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.Test;
    +import scala.tools.cmd.gen.AnyVals;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +
    +	public static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	public static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	public static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	public static final byte[] F_1 = Bytes.toBytes("f1");
    +	public static final byte[] Q_1 = Bytes.toBytes("q1");
    +	public static final byte[] Q_2 = Bytes.toBytes("q2");
    +	public static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@Test
    +	public void testHBaseTableSource() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test");
    +		createTable(tableName, F_1, new byte[1][]);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_2, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_2, Bytes.toBytes(19992l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_2, Bytes.toBytes(19993l));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		String[] colNames = new String[3];
    +		colNames[0] = Bytes.toString(F_1)+":"+Bytes.toString(Q_1);
    --- End diff --
    
    could you execute `mvn clean verify -DskipTests` to see syntax mistakes?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    You need to recompile `TableSource` trait manually and implement `DefinedFieldNames` in `HBaseTableSource`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97699488
  
    --- 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()]);
    --- End diff --
    
    I would like to move these code into `getReturnType()`, the schema may change after construction of `HBaseTableSource`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97015267
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
    +import org.apache.flink.api.common.io.RichInputFormat;
    +import org.apache.flink.api.common.io.statistics.BaseStatistics;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.io.InputSplitAssigner;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.client.Table;
    +import org.apache.hadoop.hbase.client.ClusterConnection;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.ResultScanner;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.HRegionLocator;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends RichInputFormat<Row, TableInputSplit> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private TypeInformation[] fieldTypeInfos;
    +	private String[] fieldNames;
    +	private transient Table table;
    +	private transient Scan scan;
    +	private transient Connection conn;
    +	private ResultScanner resultScanner = null;
    +
    +	private byte[] lastRow;
    +	private int scannedRows;
    +	private boolean endReached = false;
    +	private org.apache.hadoop.conf.Configuration conf;
    +	private static final String COLON = ":";
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, String[] fieldNames, TypeInformation[] fieldTypeInfos) {
    +		this.conf = conf;
    +		this.tableName = tableName;
    +		this.fieldNames = fieldNames;
    +		this.fieldTypeInfos = fieldTypeInfos;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = createScanner();
    +		}
    +	}
    +
    +	private Scan createScanner() {
    +		Scan scan = new Scan();
    +		for(String field : fieldNames) {
    +			// select only the fields in the 'selectedFields'
    +			String[] famCol = field.split(COLON);
    +			scan.addColumn(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +		}
    +		return scan;
    +	}
    +
    +	private void connectToTable() {
    +		//use files found in the classpath
    +		if(this.conf == null) {
    +			this.conf = HBaseConfiguration.create();
    +		}
    +		try {
    +			conn = ConnectionFactory.createConnection(this.conf);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while creating connection to hbase cluster", ioe);
    +			return;
    +		}
    +		try {
    +			table = conn.getTable(TableName.valueOf(tableName));
    +		} catch(TableNotFoundException tnfe) {
    +			LOG.error("The table " + tableName + " not found ", tnfe);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while connecting to the table "+tableName+ " ", ioe);
    +		}
    +	}
    +
    +	@Override
    +	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
    +		return null;
    +	}
    +
    +	@Override
    +	public TableInputSplit[] createInputSplits(final int minNumSplits) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +
    +		//Gets the starting and ending row keys for every region in the currently open table
    +		HRegionLocator regionLocator = new HRegionLocator(table.getName(), (ClusterConnection) conn);
    +		final Pair<byte[][], byte[][]> keys = regionLocator.getStartEndKeys();
    +		if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
    +			throw new IOException("Expecting at least one region.");
    +		}
    +		final byte[] startRow = scan.getStartRow();
    +		final byte[] stopRow = scan.getStopRow();
    +		final boolean scanWithNoLowerBound = startRow.length == 0;
    +		final boolean scanWithNoUpperBound = stopRow.length == 0;
    +
    +		final List<TableInputSplit> splits = new ArrayList<TableInputSplit>(minNumSplits);
    +		for (int i = 0; i < keys.getFirst().length; i++) {
    +			final byte[] startKey = keys.getFirst()[i];
    +			final byte[] endKey = keys.getSecond()[i];
    +			final String regionLocation = regionLocator.getRegionLocation(startKey, false).getHostnamePort();
    +			//Test if the given region is to be included in the InputSplit while splitting the regions of a table
    +			if (!includeRegionInSplit(startKey, endKey)) {
    +				continue;
    +			}
    +			//Finds the region on which the given row is being served
    +			final String[] hosts = new String[]{regionLocation};
    +
    +			// determine if regions contains keys used by the scan
    +			boolean isLastRegion = endKey.length == 0;
    +			if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0) &&
    +				(scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) {
    +
    +				final byte[] splitStart = scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0 ? startKey : startRow;
    +				final byte[] splitStop = (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0)
    +					&& !isLastRegion ? endKey : stopRow;
    +				int id = splits.size();
    +				final TableInputSplit split = new TableInputSplit(id, hosts, table.getName().getName(), splitStart, splitStop);
    +				splits.add(split);
    +			}
    +		}
    +		LOG.info("Created " + splits.size() + " splits");
    +		for (TableInputSplit split : splits) {
    +			logSplitInfo("created", split);
    +		}
    +		return splits.toArray(new TableInputSplit[0]);
    +	}
    +
    +	protected boolean includeRegionInSplit(final byte[] startKey, final byte[] endKey) {
    +		return true;
    +	}
    +
    +	@Override
    +	public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
    +		return new LocatableInputSplitAssigner(inputSplits);
    +	}
    +
    +	@Override
    +	public void open(TableInputSplit split) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +		if (split == null) {
    +			throw new IOException("Input split is null!");
    +		}
    +
    +		logSplitInfo("opening", split);
    +		// set the start row and stop row from the splits
    +		scan.setStartRow(split.getStartRow());
    +		lastRow = split.getEndRow();
    +		scan.setStopRow(lastRow);
    +
    +		resultScanner = table.getScanner(scan);
    +		endReached = false;
    +		scannedRows = 0;
    +	}
    +
    +	private void logSplitInfo(String action, TableInputSplit split) {
    +		int splitId = split.getSplitNumber();
    +		String splitStart = Bytes.toString(split.getStartRow());
    +		String splitEnd = Bytes.toString(split.getEndRow());
    +		String splitStartKey = splitStart.isEmpty() ? "-" : splitStart;
    +		String splitStopKey = splitEnd.isEmpty() ? "-" : splitEnd;
    +		String[] hostnames = split.getHostnames();
    +		LOG.info("{} split (this={})[{}|{}|{}|{}]", action, this, splitId, hostnames, splitStartKey, splitStopKey);
    +	}
    +
    +	@Override
    +	public boolean reachedEnd() throws IOException {
    +		return endReached;
    +	}
    +
    +	@Override
    +	public Row nextRecord(Row reuse) throws IOException {
    +		if (resultScanner == null) {
    +			throw new IOException("No table result scanner provided!");
    +		}
    +		try {
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		} catch (Exception e) {
    +			resultScanner.close();
    +			//workaround for timeout on scan
    +			LOG.warn("Error after scan of " + scannedRows + " rows. Retry with a new scanner...", e);
    +			scan.setStartRow(lastRow);
    +			resultScanner = table.getScanner(scan);
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		}
    +		endReached = true;
    +		return null;
    +	}
    +
    +	private Row mapResultToRow(Result res) {
    +		Object[] values = new Object[fieldNames.length];
    +		int i = 0;
    +		for(String field : fieldNames) {
    +			String[] famCol = field.split(COLON);
    +			byte[] value = res.getValue(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +			TypeInformation typeInfo = fieldTypeInfos[i];
    +			if(typeInfo.isBasicType()) {
    +				if(typeInfo.getTypeClass() ==  Integer.class) {
    +					values[i] = Bytes.toInt(value);
    +				} else if(typeInfo.getTypeClass() == Short.class) {
    +					values[i] = Bytes.toShort(value);
    +				} else if(typeInfo.getTypeClass() == Float.class) {
    +					values[i] = Bytes.toFloat(value);
    +				} else if(typeInfo.getTypeClass() == Long.class) {
    +					values[i] = Bytes.toLong(value);
    +				} else if(typeInfo.getTypeClass() == String.class) {
    +					values[i] = Bytes.toString(value);
    +				} else if(typeInfo.getTypeClass() == Byte.class) {
    +					values[i] = value[0];
    +				} else if(typeInfo.getTypeClass() == Boolean.class) {
    +					values[i] = Bytes.toBoolean(value);
    +				} else if(typeInfo.getTypeClass() == Double.class) {
    +					values[i] = Bytes.toDouble(value);
    +				} else if(typeInfo.getTypeClass() == BigInteger.class) {
    +					values[i] = new BigInteger(value);
    +				} else if(typeInfo.getTypeClass() == BigDecimal.class) {
    +					values[i] = Bytes.toBigDecimal(value);
    +				} else if(typeInfo.getTypeClass() == Date.class) {
    +					values[i] = new Date(Bytes.toLong(value));
    +				}
    +			} else {
    +				// TODO for other types??
    +			}
    +			i++;
    +		}
    +		return Row.of(values);
    +	}
    +
    +	@Override
    +	public void close() throws IOException {
    +		LOG.info("Closing split (scanned {} rows)", scannedRows);
    +		lastRow = null;
    +		try {
    +			if (resultScanner != null) {
    +				resultScanner.close();
    +			}
    +		} finally {
    +			resultScanner = null;
    +		}
    +	}
    +
    +	@Override
    +	public void closeInputFormat() throws IOException {
    +		try {
    +			if (table != null) {
    +				table.close();
    +			}
    +		} finally {
    +			table = null;
    +		}
    +	}
    +
    +	@Override
    +	public TypeInformation<Row> getProducedType() {
    +		return new RowTypeInfo(this.fieldTypeInfos);
    --- End diff --
    
    Ok. Got it.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Hi @ramkrish86, @tonycox, and @wuchong,
    
    sorry for joining the discussion a bit late. I haven't looked at the code yet, but I think the discussion is going into the right direction. 
    
    I had a look at [how Apache Drill provides access to HBase tables](https://drill.apache.org/docs/querying-hbase/). Drill also uses a nested schema of `[rowkey, colfamily1[col1, col2, ...], colfamiliy2[col1, col2, ...] ...]` so basically the same as we are discussing here.
    
    Regarding the field types: The serialization is not under our control, so should also offer to just return the raw bytes (as Drill does). If users have custom data types or serialization logic they can use a user defined scalar function to extract the value. I don't know what's the standard serialization format for primitives with HBase (or if there is one at all). 
    
    Regarding restricting the scan with rowkeys. @tonycox's PR for [filterable TableSources](https://github.com/apache/flink/pull/3166) can be used to set the scan range. This would be much better than "hardcoding" the scan ranges in the TableSource.
    
    Best, Fabian


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98805674
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java ---
    @@ -41,7 +41,7 @@
     /**
      * {@link InputFormat} subclass that wraps the access for HTables.
      */
    -public abstract class TableInputFormat<T extends Tuple> extends RichInputFormat<T, TableInputSplit> {
    +public abstract class TableInputFormat<T> extends RichInputFormat<T, TableInputSplit> {
    --- End diff --
    
    I'd rather rename this class to `AbstractTableInputFormat<T>` and add a new class `TableInputFormat<T extends Tuple>` to keep the API stable.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98145293
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,196 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSchema;
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.test.util.TestBaseUtils;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +	private static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	private static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	private static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	private static final byte[] F_1 = Bytes.toBytes("f1");
    +	private static final byte[] F_2 = Bytes.toBytes("f2");
    +	private static final byte[] Q_1 = Bytes.toBytes("q1");
    +	private static final byte[] Q_2 = Bytes.toBytes("q2");
    +	private static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@BeforeClass
    +	public static void activateHBaseCluster(){
    +		registerHBaseMiniClusterInClasspath();
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithSingleColumnFamily() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test");
    +		// no split keys
    +		byte[][] famNames = new byte[1][];
    +		famNames[0] = F_1;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19992L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993L));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		HBaseTableSchema schema = new HBaseTableSchema();
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_3), Long.class);
    +		// fetch row2 from the table till the end
    +		BatchTableSource hbaseTable = new HBaseTableSource(getConf(), tableName.getNameAsString(), schema);
    +		tableEnv.registerTableSource("test", hbaseTable);
    +		Table result = tableEnv
    +			.sql("SELECT test.f1.q1, test.f1.q2, test.f1.q3 FROM test");
    +		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    +		List<Row> results = resultSet.collect();
    +
    +		String expected = "100,strvalue,19991\n" +
    +			"101,strvalue1,19992\n" +
    +			"102,strvalue2,19993\n";
    +		TestBaseUtils.compareResultAsText(results, expected);
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithTwoColumnFamily() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test1");
    +		// no split keys
    +		byte[][] famNames = new byte[2][];
    +		famNames[0] = F_1;
    +		famNames[1] = F_2;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_2, Q_1, Bytes.toBytes(201));
    +		//2nd qual is String
    +		put.addColumn(F_2, Q_2, Bytes.toBytes("newvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_2, Q_3, Bytes.toBytes(29992L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993L));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, 	new TableConfig());
    +		HBaseTableSchema schema = new HBaseTableSchema();
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_1), Bytes.toString(Q_3), Long.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_1), Integer.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_2), String.class);
    +		schema.addColumn(Bytes.toString(F_2), Bytes.toString(Q_3), Long.class);
    +		// fetch row2 from the table till the end
    +		BatchTableSource hbaseTable = new HBaseTableSource(getConf(), tableName.getNameAsString(), schema);
    +		tableEnv.registerTableSource("test1", hbaseTable);
    +		Table result = tableEnv
    +			.sql("SELECT test1.f1.q1, test1.f1.q2, test1.f1.q3, test1.f2.q1, test1.f2.q2, test1.f2.q3 FROM test1");
    +		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    +		List<Row> results = resultSet.collect();
    +
    +		String expected = "100,strvalue,19991,null,null,null\n" +
    +			"null,null,null,201,newvalue1,29992\n" +
    +			"102,strvalue2,19993,null,null,null\n";
    +		TestBaseUtils.compareResult(results, expected, false, false);
    --- End diff --
    
    Actually this API was expecting things to be sorted. I did not want that to happen. Ok then will ensure that the assertion is done after sorting. Thanks for all the comments. Will update the PR shortly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98832296
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends TableInputFormat<Row> implements ResultTypeQueryable<Row> {
    --- End diff --
    
    This class could also be used without the `TableSource` from a regular DataSet program. I think the name `HBaseRowInputFormat` would make that more clear and in addition indicate that it returns `Row` objects.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    bq.ProjectableTableSource works in scan process. 
    Ya got it. I was just trying to relate with this HBase thing and could find that we try to read all cols and then do a flatMap and then return the required cols alone. Just read that PR to understand better.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97074517
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java ---
    @@ -0,0 +1,75 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +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;
    +
    +/**
    + * 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> {
    +
    +	private Configuration conf;
    +	private String tableName;
    +	private byte[] rowKey;
    +	private String[] colNames;
    +	private TypeInformation<?>[] colTypes;
    +
    +	public HBaseTableSource(Configuration conf, String tableName, byte[] rowKey, String[] colNames,
    +							TypeInformation<?>[] colTypes) {
    +		this.conf = conf;
    +		this.tableName = Preconditions.checkNotNull(tableName, "Table  name");
    +		this.rowKey = Preconditions.checkNotNull(rowKey, "Rowkey");
    --- End diff --
    
    ` f0~fn-1 `


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    > Regarding the HBaseTableSchema, we could also use it only internally and not expose it to the user. The HBaseTableSource would have a method addColumn() and forward the calls to its internal HBaseSchema.
    
    Have done this. I initially thought to do things in the construction time itself. Now added an addColumn() in hbaseTableSource. HBaseSchema becomes totally package private and no access to users.
    
    Regarding the flatSchema, generally in hbase only family is required and the qualifiers are just dynamic. But here for the sake of accessibility we expect the user to specify the column names. (trying to give a relational look). It is in case of projections is where we have some issues. In my opinion if we have a better API for Projection may be we could handle it better? The current nested way as you said is better in the sense that all columns of a family are grouped together. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Alright. The PR looks mostly good. I'll make a few refinements and will merge it then.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97709590
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -19,99 +19,113 @@
     package org.apache.flink.addons.hbase;
     
     import org.apache.flink.api.common.io.InputFormat;
    -import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
    -import org.apache.flink.api.common.io.RichInputFormat;
    -import org.apache.flink.api.common.io.statistics.BaseStatistics;
     import org.apache.flink.api.common.typeinfo.TypeInformation;
     import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
     import org.apache.flink.api.java.typeutils.RowTypeInfo;
     import org.apache.flink.configuration.Configuration;
    -import org.apache.flink.core.io.InputSplitAssigner;
     import org.apache.flink.types.Row;
     import org.apache.hadoop.hbase.HBaseConfiguration;
     import org.apache.hadoop.hbase.TableName;
     import org.apache.hadoop.hbase.TableNotFoundException;
    -import org.apache.hadoop.hbase.client.Scan;
    -import org.apache.hadoop.hbase.client.Table;
    -import org.apache.hadoop.hbase.client.ClusterConnection;
    -import org.apache.hadoop.hbase.client.Result;
    -import org.apache.hadoop.hbase.client.ResultScanner;
    -import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.HTable;
     import org.apache.hadoop.hbase.client.Connection;
    -import org.apache.hadoop.hbase.client.HRegionLocator;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.Scan;
     import org.apache.hadoop.hbase.util.Bytes;
     import org.apache.hadoop.hbase.util.Pair;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.math.BigDecimal;
    -import java.math.BigInteger;
     import java.util.ArrayList;
    -import java.util.Date;
     import java.util.List;
    +import java.util.Map;
     
     /**
      * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
      */
    -public class HBaseTableSourceInputFormat extends RichInputFormat<Row, TableInputSplit> implements ResultTypeQueryable<Row> {
    +public class HBaseTableSourceInputFormat extends TableInputFormat<Row> implements ResultTypeQueryable<Row> {
     
     	private static final long serialVersionUID = 1L;
     
     	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
     	private String tableName;
    -	private TypeInformation[] fieldTypeInfos;
    -	private String[] fieldNames;
    -	private transient Table table;
    -	private transient Scan scan;
     	private transient Connection conn;
    -	private ResultScanner resultScanner = null;
    -
    -	private byte[] lastRow;
    -	private int scannedRows;
    -	private boolean endReached = false;
    -	private org.apache.hadoop.conf.Configuration conf;
    -	private static final String COLON = ":";
    +	private transient org.apache.hadoop.conf.Configuration conf;
    +	private HBaseTableSchema schema;
     
    -	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, String[] fieldNames, TypeInformation[] fieldTypeInfos) {
    -		this.conf = conf;
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, HBaseTableSchema schema) {
     		this.tableName = tableName;
    -		this.fieldNames = fieldNames;
    -		this.fieldTypeInfos = fieldTypeInfos;
    +		this.conf = conf;
    +		this.schema = schema;
     	}
     
     	@Override
     	public void configure(Configuration parameters) {
     		LOG.info("Initializing HBaseConfiguration");
     		connectToTable();
     		if(table != null) {
    -			scan = createScanner();
    +			scan = getScanner();
     		}
     	}
     
    -	private Scan createScanner() {
    +	@Override
    +	protected Scan getScanner() {
    +		// TODO : Pass 'rowkey'. For this we need FilterableTableSource
     		Scan scan = new Scan();
    -		for(String field : fieldNames) {
    +		Map<String, List<Pair>> familyMap = schema.getFamilyMap();
    +		for(String family : familyMap.keySet()) {
     			// select only the fields in the 'selectedFields'
    -			String[] famCol = field.split(COLON);
    -			scan.addColumn(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +			List<Pair> colDetails = familyMap.get(family);
    +			for(Pair<String, TypeInformation<?>> pair : colDetails) {
    +				scan.addColumn(Bytes.toBytes(family), Bytes.toBytes(pair.getFirst()));
    +			}
     		}
     		return scan;
     	}
     
    +	@Override
    +	public String getTableName() {
    +		return tableName;
    +	}
    +
    +	@Override
    +	protected Row mapResultToTuple(Result res) {
    +		List<Object> values = new ArrayList<Object>();
    +		int i = 0;
    +		Map<String, List<Pair>> familyMap = schema.getFamilyMap();
    +		Row[] rows = new Row[familyMap.size()];
    --- End diff --
    
    Will check this too. Thanks for all the nice comments @wuchong .


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    I tried multiple ways to take the dump file from this mvn test command run. I get a hprof file which on opening in heap dump analyser throws EOF exception or NPE exception. 
    @tonycox - were you able to get any heap dump?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97955449
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,140 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.sql.Time;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.sql.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    +		new HashMap<>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class, Time.class, byte[].class
    +	};
    +	/**
    +	 * Allows specifying the family and qualifier name along with the data type of the qualifier for an HBase table
    +	 *
    +	 * @param family    the family name
    +	 * @param qualifier the qualifier name
    +	 * @param clazz     the data type of the qualifier
    +	 */
    +	public void addColumn(String family, String qualifier, Class<?> clazz) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(qualifier, "qualifier name");
    +		Preconditions.checkNotNull(clazz, "class type");
    +		List<Pair<String, TypeInformation<?>>> list = this.familyMap.get(family);
    +		if (list == null) {
    +			list = new ArrayList<>();
    +		}
    +		boolean found = false;
    +		for (Class classType : CLASS_TYPES) {
    +			if (classType == clazz) {
    +				found = true;
    +				break;
    +			}
    +		}
    +		if (!found) {
    +			// throw exception
    +			throw new IllegalArgumentException("Unsupported class type found " + clazz);
    +		}
    +		list.add(new Pair<String, TypeInformation<?>>(qualifier, TypeExtractor.getForClass(clazz)));
    +		familyMap.put(family, list);
    +	}
    +
    +	public String[] getFamilyNames() {
    +		return this.familyMap.keySet().toArray(new String[this.familyMap.size()]);
    +	}
    +
    +	public String[] getQualifierNames(String family) {
    +		List<Pair<String, TypeInformation<?>>> colDetails = familyMap.get(family);
    +		String[] qualifierNames = new String[colDetails.size()];
    +		int i = 0;
    +		for (Pair<String, TypeInformation<?>> pair : colDetails) {
    +			qualifierNames[i] = pair.getFirst();
    +			i++;
    +		}
    +		return qualifierNames;
    +	}
    +
    +	public TypeInformation<?>[] getTypeInformation(String family) {
    --- End diff --
    
    Rename to `getQualifierTypes` ?  This can keep consistent to `getQualifierNames`. The `getTypeInformation` is a little mislead IMO.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @fhueske ,
    I have fixed the comments as per @wuchong . And he has said +1 after fixing them. Would like to see the PR and see if it is fine with you too.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @fhueske - Please have a look at the javadoc.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Thanks for all the inputs here. I have been trying to make my existing code work with the composite RowTypeInfo. Once that is done I will try to introduce the HBaseTableSchema.
    Also I would like to work on FLINK-3849 (FilterableTableSource) after this first version of HBaseTableSource is accepted.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97903962
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,248 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSchema;
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +	public static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	public static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	public static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	public static final byte[] F_1 = Bytes.toBytes("f1");
    +	public static final byte[] F_2 = Bytes.toBytes("f2");
    +	public static final byte[] Q_1 = Bytes.toBytes("q1");
    +	public static final byte[] Q_2 = Bytes.toBytes("q2");
    +	public static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@BeforeClass
    +	public static void activateHBaseCluster(){
    +		registerHBaseMiniClusterInClasspath();
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithSingleColumnFamily() throws Exception {
    +		// create a table with single region
    +		MapFunction<Row, String> mapFunction = new MapFunction<Row, String>() {
    +
    +			@Override
    +			public String map(Row value) throws Exception {
    +				return value == null ? "null" : value.toString();
    +			}
    +		};
    +		TableName tableName = TableName.valueOf("test");
    +		// no split keys
    +		byte[][] famNames = new byte[1][];
    +		famNames[0] = F_1;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19992l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993l));
    --- End diff --
    
    could you change it to upper L?
    low l sometimes looks like 1


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Updated the PR fixing the comments. The comments were simple but the adding AbstractTableInputFormat and moving the code back and forth makes this one a bigger change. But internally they are just refactorings. The test cases passes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    It is CentOS Linux release 7.0.1406 (Core). 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98805386
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends TableInputFormat<Row> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private transient Connection conn;
    +	private transient org.apache.hadoop.conf.Configuration conf;
    +	private HBaseTableSchema schema;
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, HBaseTableSchema schema) {
    +		this.tableName = tableName;
    +		this.conf = conf;
    +		this.schema = schema;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = getScanner();
    +		}
    +	}
    +
    +	@Override
    +	protected Scan getScanner() {
    +		Scan scan = new Scan();
    +		for(String family : schema.getFamilyNames()) {
    +			for(String qualifierName : schema.getQualifierNames(family)) {
    --- End diff --
    
    Is the order of qualifier names guaranteed?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @fhueske , @tonycox , @wuchong 
    I have updated the PR based on all the feedbacks here. Now you could see that we now support CompoisteRowType and we are able to specify multiple column families along with the qualifier names.
    We are able to retrieve the result by doing a full scan.
    This is not efficient and we need to specify start and end rows. I think that can be done after FilterableTableSource is done.
    I have added test cases that shows single column family and double column family. 
    For now if the TypeInformation is not known we use plain byte[] type only. That happens at the validation state itself. But one main concern from my side is how to present the 'NULL' means we specify a column with a type but there is no data for that column. For now I have handled by returning the Int, Float, Long - Min_values. But that may not be right I believe. Feedback and suggestions welcome.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98805094
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends TableInputFormat<Row> implements ResultTypeQueryable<Row> {
    --- End diff --
    
    Rename to `HBaseRowInputFormat`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @fhueske I preffer a better API with nesting. We can extend `ProjectableTableSource` with a method like this
    ```scala
    def projectNestedFields(fields: Array[String]): ProjectableTableSource[T]
    ```
    with default realisation.
    override the method in HBase table source and while pushing projection down extract `fieldAccessor`s from `inputRex` and push them into as well.
    Or create another `ProjectableNestedFieldsTableSource` with the same method.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98828283
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java ---
    @@ -0,0 +1,83 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +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 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 ':'
    --- End diff --
    
    True. Will remove.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98435113
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,140 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.sql.Time;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.sql.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    +		new HashMap<>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class, Time.class, byte[].class
    +	};
    +	/**
    +	 * Allows specifying the family and qualifier name along with the data type of the qualifier for an HBase table
    +	 *
    +	 * @param family    the family name
    +	 * @param qualifier the qualifier name
    +	 * @param clazz     the data type of the qualifier
    +	 */
    +	public void addColumn(String family, String qualifier, Class<?> clazz) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(qualifier, "qualifier name");
    +		Preconditions.checkNotNull(clazz, "class type");
    +		List<Pair<String, TypeInformation<?>>> list = this.familyMap.get(family);
    +		if (list == null) {
    +			list = new ArrayList<>();
    +		}
    +		boolean found = false;
    +		for (Class classType : CLASS_TYPES) {
    +			if (classType == clazz) {
    +				found = true;
    +				break;
    +			}
    +		}
    +		if (!found) {
    --- End diff --
    
    and just `if (!CLASS_TYPES.contains(clazz))` here


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    I think it can be solved in different issue to provide a new api in `table.api` for selecting from HBase


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97067569
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java ---
    @@ -0,0 +1,75 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +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;
    +
    +/**
    + * 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> {
    +
    +	private Configuration conf;
    +	private String tableName;
    +	private byte[] rowKey;
    +	private String[] colNames;
    +	private TypeInformation<?>[] colTypes;
    +
    +	public HBaseTableSource(Configuration conf, String tableName, byte[] rowKey, String[] colNames,
    +							TypeInformation<?>[] colTypes) {
    +		this.conf = conf;
    +		this.tableName = Preconditions.checkNotNull(tableName, "Table  name");
    +		this.rowKey = Preconditions.checkNotNull(rowKey, "Rowkey");
    --- End diff --
    
    Try to use `.sql("SELECT test.f1.q1, test.f1.q2t")` or table api instead. I think there is a problem with nested Rows while scaning


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97738061
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair>> familyMap =
    +		new HashMap<String, List<Pair>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class
    +	};
    +	private static byte[] EMPTY_BYTE_ARRAY = new byte[0];
    +	public void addColumns(String family, String qualifier, TypeInformation<?> type) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(family, "qualifier name");
    +		Preconditions.checkNotNull(type, "type name");
    +		List<Pair> list = this.familyMap.get(family);
    +		if (list == null) {
    +			list = new ArrayList<Pair>();
    +		}
    +		boolean found = false;
    +		for(Class classType : CLASS_TYPES) {
    +			if(classType == type.getTypeClass()) {
    +				found = true;
    +				break;
    +			}
    +		}
    +		if(!found) {
    +			// by default it will be byte[] type only
    +			type = BasicArrayTypeInfo.BYTE_ARRAY_TYPE_INFO;
    --- End diff --
    
    Ok I will add byte[].class into the CLASS_TYPES. And anything other than the ones in CLASS_TYPES we can throw an exception.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Thanks for the ping here @tonycox . 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98828345
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends TableInputFormat<Row> implements ResultTypeQueryable<Row> {
    --- End diff --
    
    Here the Row - you mean the Type of generic that is passed to this class? Because the HBase table also has the concept of row which we are now not using it. So I feel the current name is better. What you think?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98804629
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java ---
    @@ -0,0 +1,83 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +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 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 ':'
    --- End diff --
    
    I think this comment is no longer valid as columns are now nested.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @ramkrish86 sorry for confusing, don't pay attention to new generating maps.
    I think `PushProjectIntoBatchTableSourceScanRule` is not good enough for nested data types. but we can project at least family columns now. 
    `HBaseTableSchema` keep `familyMap` and might keep a list with family names.
    for example, while `addColumn`
    ```java
    List<Pair<String, TypeInformation<?>>> list = this.familyMap.get(family);
        if (list == null) {
            familyNames.add(family);
            list = new ArrayList<>();
        }
    ```
    so we have access to family column by index


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Even jhat was not able to view the file as it had a problem in parsing the hprof file. So my question is if MaxPermSize is 128M - why does it work with jdk 8? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97304540
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java ---
    @@ -0,0 +1,75 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +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;
    +
    +/**
    + * 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> {
    +
    +	private Configuration conf;
    +	private String tableName;
    +	private byte[] rowKey;
    +	private String[] colNames;
    +	private TypeInformation<?>[] colTypes;
    +
    +	public HBaseTableSource(Configuration conf, String tableName, byte[] rowKey, String[] colNames,
    +							TypeInformation<?>[] colTypes) {
    +		this.conf = conf;
    +		this.tableName = Preconditions.checkNotNull(tableName, "Table  name");
    +		this.rowKey = Preconditions.checkNotNull(rowKey, "Rowkey");
    --- End diff --
    
    I tried this customization but still if I pass something like f1.q1 it was throwing a validation error which was fine after I used it as suggested by @tonycox .


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Fixed all the minor comments given above. @tonycox , @wuchong , @fhueske .


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    I tried even that. The test runs fine for me. No OOME I get.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r96665689
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,117 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.Test;
    +import scala.tools.cmd.gen.AnyVals;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +
    +	public static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	public static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	public static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	public static final byte[] F_1 = Bytes.toBytes("f1");
    +	public static final byte[] Q_1 = Bytes.toBytes("q1");
    +	public static final byte[] Q_2 = Bytes.toBytes("q2");
    +	public static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@Test
    +	public void testHBaseTableSource() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test");
    +		createTable(tableName, F_1, new byte[1][]);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_2, Bytes.toBytes(19991l));
    --- End diff --
    
    I think there should be `Q_3`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    > Exception: java.lang.OutOfMemoryError thrown from the UncaughtExceptionHandler in thread "RpcServer.reader=6,bindAddress=testing-docker-bb4f2e37-e79f-42a3-a9e9-4995e42c70ba,port=45919"
    Exception: java.lang.OutOfMemoryError thrown from the UncaughtExceptionHandler in thread "CHAIN DataSource (at getDataSet(HBaseTableSource.java:63) (org.apache.flink.addons.hbase.HBaseTableSourceInputFormat)) -> FlatMap (select: (f1.q1 AS q1, f1.q2 AS q2, f1.q3 AS q3)) (6/32)"
    01/27/2017 05:57:12	DataSink (collect())(13/32) switched to DEPLOYING 
    Exception: java.lang.OutOfMemoryError thrown from the UncaughtExceptionHandler in thread "LeaseRenewer:travis@localhost:39289"
    01/27/2017 05:57:14	DataSink (collect())(12/32) switched to SCHEDULED 
    Exception: java.lang.OutOfMemoryError thrown from the UncaughtExceptionHandler in thread "CHAIN DataSource (at getDataSet(HBaseTableSource.java:63) (org.apache.flink.addons.hbase.HBaseTableSourceInputFormat)) -> FlatMap (select: (f1.q1 AS q1, f1.q2 AS q2, f1.q3 AS q3)) (10/32)"
    
    Getting this error in the travis build. But in my linux box it seems to pass.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @fhueske  - so are you ok with @tonycox  suggestion of setting MAxPermSize for hbase module?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    `HBaseTableSchema` is initial information of table columns, so we can add more maps that generate after familyMap has set
    ```java
    Map<String, List<Pair<String, TypeInformation<?>>>> familyMap;
    Map<String, Integer> familyMapId;
    Map<String, Integer> columnMapId;
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @fhueske , @tonycox , @wuchong - FYI.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97954670
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,140 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.sql.Time;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.sql.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    +		new HashMap<>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class, Time.class, byte[].class
    +	};
    +	/**
    +	 * Allows specifying the family and qualifier name along with the data type of the qualifier for an HBase table
    +	 *
    +	 * @param family    the family name
    +	 * @param qualifier the qualifier name
    +	 * @param clazz     the data type of the qualifier
    +	 */
    +	public void addColumn(String family, String qualifier, Class<?> clazz) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(qualifier, "qualifier name");
    +		Preconditions.checkNotNull(clazz, "class type");
    +		List<Pair<String, TypeInformation<?>>> list = this.familyMap.get(family);
    +		if (list == null) {
    +			list = new ArrayList<>();
    +		}
    +		boolean found = false;
    +		for (Class classType : CLASS_TYPES) {
    +			if (classType == clazz) {
    +				found = true;
    +				break;
    +			}
    +		}
    +		if (!found) {
    +			// throw exception
    +			throw new IllegalArgumentException("Unsupported class type found " + clazz);
    --- End diff --
    
    I think we can improve the error message to suggest users to use `byte[].class` instead and deserialize the byte array using user-defined scalar function.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @wuchong 
    I think I have updated the last comments from you. Thank you for all your help/support here.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ex00 <gi...@git.apache.org>.
Github user ex00 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97730063
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair>> familyMap =
    +		new HashMap<String, List<Pair>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class
    +	};
    +	private static byte[] EMPTY_BYTE_ARRAY = new byte[0];
    +	public void addColumns(String family, String qualifier, TypeInformation<?> type) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(family, "qualifier name");
    +		Preconditions.checkNotNull(type, "type name");
    +		List<Pair> list = this.familyMap.get(family);
    +		if (list == null) {
    +			list = new ArrayList<Pair>();
    +		}
    +		boolean found = false;
    +		for(Class classType : CLASS_TYPES) {
    +			if(classType == type.getTypeClass()) {
    +				found = true;
    +				break;
    +			}
    +		}
    +		if(!found) {
    +			// by default it will be byte[] type only
    +			type = BasicArrayTypeInfo.BYTE_ARRAY_TYPE_INFO;
    +		}
    +		list.add(new Pair(qualifier, type));
    +		familyMap.put(family, list);
    +	}
    +
    +	public Map<String, List<Pair>> getFamilyMap() {
    +		return this.familyMap;
    +	}
    +
    +	public Object deserialize(byte[] value, TypeInformation<?> typeInfo) {
    +		if (typeInfo.isBasicType()) {
    +			if (typeInfo.getTypeClass() == Integer.class) {
    +				return Bytes.toInt(value);
    +			} else if (typeInfo.getTypeClass() == Short.class) {
    +				return Bytes.toShort(value);
    +			} else if (typeInfo.getTypeClass() == Float.class) {
    +				return Bytes.toFloat(value);
    +			} else if (typeInfo.getTypeClass() == Long.class) {
    +				return Bytes.toLong(value);
    +			} else if (typeInfo.getTypeClass() == String.class) {
    +				return Bytes.toString(value);
    +			} else if (typeInfo.getTypeClass() == Byte.class) {
    +				return value[0];
    +			} else if (typeInfo.getTypeClass() == Boolean.class) {
    +				return Bytes.toBoolean(value);
    +			} else if (typeInfo.getTypeClass() == Double.class) {
    +				return Bytes.toDouble(value);
    +			} else if (typeInfo.getTypeClass() == BigInteger.class) {
    +				return new BigInteger(value);
    +			} else if (typeInfo.getTypeClass() == BigDecimal.class) {
    +				return Bytes.toBigDecimal(value);
    +			} else if (typeInfo.getTypeClass() == Date.class) {
    +				return new Date(Bytes.toLong(value));
    +			}
    +		}
    +		return value;
    +	}
    +
    +	public Object deserializeNull(TypeInformation<?> typeInfo) {
    +		// TODO : this may need better handling.
    +		if(typeInfo.getTypeClass() ==  Integer.class) {
    --- End diff --
    
    Why don't try use here ``org.apache.flink.table.codegen.CodeGenUtils#primitiveDefaultValue``  for basic types and ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r96802576
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
    +import org.apache.flink.api.common.io.RichInputFormat;
    +import org.apache.flink.api.common.io.statistics.BaseStatistics;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.io.InputSplitAssigner;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.client.Table;
    +import org.apache.hadoop.hbase.client.ClusterConnection;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.ResultScanner;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.HRegionLocator;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends RichInputFormat<Row, TableInputSplit> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private TypeInformation[] fieldTypeInfos;
    +	private String[] fieldNames;
    +	private transient Table table;
    +	private transient Scan scan;
    +	private transient Connection conn;
    +	private ResultScanner resultScanner = null;
    +
    +	private byte[] lastRow;
    +	private int scannedRows;
    +	private boolean endReached = false;
    +	private org.apache.hadoop.conf.Configuration conf;
    +	private static final String COLON = ":";
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, String[] fieldNames, TypeInformation[] fieldTypeInfos) {
    +		this.conf = conf;
    +		this.tableName = tableName;
    +		this.fieldNames = fieldNames;
    +		this.fieldTypeInfos = fieldTypeInfos;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = createScanner();
    +		}
    +	}
    +
    +	private Scan createScanner() {
    +		Scan scan = new Scan();
    +		for(String field : fieldNames) {
    +			// select only the fields in the 'selectedFields'
    +			String[] famCol = field.split(COLON);
    +			scan.addColumn(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +		}
    +		return scan;
    +	}
    +
    +	private void connectToTable() {
    +		//use files found in the classpath
    +		if(this.conf == null) {
    +			this.conf = HBaseConfiguration.create();
    +		}
    +		try {
    +			conn = ConnectionFactory.createConnection(this.conf);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while creating connection to hbase cluster", ioe);
    +			return;
    +		}
    +		try {
    +			table = conn.getTable(TableName.valueOf(tableName));
    +		} catch(TableNotFoundException tnfe) {
    +			LOG.error("The table " + tableName + " not found ", tnfe);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while connecting to the table "+tableName+ " ", ioe);
    +		}
    +	}
    +
    +	@Override
    +	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
    +		return null;
    +	}
    +
    +	@Override
    +	public TableInputSplit[] createInputSplits(final int minNumSplits) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +
    +		//Gets the starting and ending row keys for every region in the currently open table
    +		HRegionLocator regionLocator = new HRegionLocator(table.getName(), (ClusterConnection) conn);
    +		final Pair<byte[][], byte[][]> keys = regionLocator.getStartEndKeys();
    +		if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
    +			throw new IOException("Expecting at least one region.");
    +		}
    +		final byte[] startRow = scan.getStartRow();
    +		final byte[] stopRow = scan.getStopRow();
    +		final boolean scanWithNoLowerBound = startRow.length == 0;
    +		final boolean scanWithNoUpperBound = stopRow.length == 0;
    +
    +		final List<TableInputSplit> splits = new ArrayList<TableInputSplit>(minNumSplits);
    +		for (int i = 0; i < keys.getFirst().length; i++) {
    +			final byte[] startKey = keys.getFirst()[i];
    +			final byte[] endKey = keys.getSecond()[i];
    +			final String regionLocation = regionLocator.getRegionLocation(startKey, false).getHostnamePort();
    +			//Test if the given region is to be included in the InputSplit while splitting the regions of a table
    +			if (!includeRegionInSplit(startKey, endKey)) {
    +				continue;
    +			}
    +			//Finds the region on which the given row is being served
    +			final String[] hosts = new String[]{regionLocation};
    +
    +			// determine if regions contains keys used by the scan
    +			boolean isLastRegion = endKey.length == 0;
    +			if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0) &&
    +				(scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) {
    +
    +				final byte[] splitStart = scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0 ? startKey : startRow;
    +				final byte[] splitStop = (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0)
    +					&& !isLastRegion ? endKey : stopRow;
    +				int id = splits.size();
    +				final TableInputSplit split = new TableInputSplit(id, hosts, table.getName().getName(), splitStart, splitStop);
    +				splits.add(split);
    +			}
    +		}
    +		LOG.info("Created " + splits.size() + " splits");
    +		for (TableInputSplit split : splits) {
    +			logSplitInfo("created", split);
    +		}
    +		return splits.toArray(new TableInputSplit[0]);
    +	}
    +
    +	protected boolean includeRegionInSplit(final byte[] startKey, final byte[] endKey) {
    +		return true;
    +	}
    +
    +	@Override
    +	public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
    +		return new LocatableInputSplitAssigner(inputSplits);
    +	}
    +
    +	@Override
    +	public void open(TableInputSplit split) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +		if (split == null) {
    +			throw new IOException("Input split is null!");
    +		}
    +
    +		logSplitInfo("opening", split);
    +		// set the start row and stop row from the splits
    +		scan.setStartRow(split.getStartRow());
    +		lastRow = split.getEndRow();
    +		scan.setStopRow(lastRow);
    +
    +		resultScanner = table.getScanner(scan);
    +		endReached = false;
    +		scannedRows = 0;
    +	}
    +
    +	private void logSplitInfo(String action, TableInputSplit split) {
    +		int splitId = split.getSplitNumber();
    +		String splitStart = Bytes.toString(split.getStartRow());
    +		String splitEnd = Bytes.toString(split.getEndRow());
    +		String splitStartKey = splitStart.isEmpty() ? "-" : splitStart;
    +		String splitStopKey = splitEnd.isEmpty() ? "-" : splitEnd;
    +		String[] hostnames = split.getHostnames();
    +		LOG.info("{} split (this={})[{}|{}|{}|{}]", action, this, splitId, hostnames, splitStartKey, splitStopKey);
    +	}
    +
    +	@Override
    +	public boolean reachedEnd() throws IOException {
    +		return endReached;
    +	}
    +
    +	@Override
    +	public Row nextRecord(Row reuse) throws IOException {
    +		if (resultScanner == null) {
    +			throw new IOException("No table result scanner provided!");
    +		}
    +		try {
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		} catch (Exception e) {
    +			resultScanner.close();
    +			//workaround for timeout on scan
    +			LOG.warn("Error after scan of " + scannedRows + " rows. Retry with a new scanner...", e);
    +			scan.setStartRow(lastRow);
    +			resultScanner = table.getScanner(scan);
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		}
    +		endReached = true;
    +		return null;
    +	}
    +
    +	private Row mapResultToRow(Result res) {
    +		Object[] values = new Object[fieldNames.length];
    +		int i = 0;
    +		for(String field : fieldNames) {
    +			String[] famCol = field.split(COLON);
    +			byte[] value = res.getValue(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +			TypeInformation typeInfo = fieldTypeInfos[i];
    +			if(typeInfo.isBasicType()) {
    +				if(typeInfo.getTypeClass() ==  Integer.class) {
    +					values[i] = Bytes.toInt(value);
    +				} else if(typeInfo.getTypeClass() == Short.class) {
    +					values[i] = Bytes.toShort(value);
    +				} else if(typeInfo.getTypeClass() == Float.class) {
    +					values[i] = Bytes.toFloat(value);
    +				} else if(typeInfo.getTypeClass() == Long.class) {
    +					values[i] = Bytes.toLong(value);
    +				} else if(typeInfo.getTypeClass() == String.class) {
    +					values[i] = Bytes.toString(value);
    +				} else if(typeInfo.getTypeClass() == Byte.class) {
    +					values[i] = value[0];
    +				} else if(typeInfo.getTypeClass() == Boolean.class) {
    +					values[i] = Bytes.toBoolean(value);
    +				} else if(typeInfo.getTypeClass() == Double.class) {
    +					values[i] = Bytes.toDouble(value);
    +				} else if(typeInfo.getTypeClass() == BigInteger.class) {
    +					values[i] = new BigInteger(value);
    +				} else if(typeInfo.getTypeClass() == BigDecimal.class) {
    +					values[i] = Bytes.toBigDecimal(value);
    +				} else if(typeInfo.getTypeClass() == Date.class) {
    +					values[i] = new Date(Bytes.toLong(value));
    +				}
    +			} else {
    +				// TODO for other types??
    --- End diff --
    
    I think you need to implement `DefinedFieldNames` trait in `HBaseTableSource`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97702519
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends TableInputFormat<Row> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private transient Connection conn;
    +	private transient org.apache.hadoop.conf.Configuration conf;
    +	private HBaseTableSchema schema;
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, HBaseTableSchema schema) {
    +		this.tableName = tableName;
    +		this.conf = conf;
    +		this.schema = schema;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = getScanner();
    +		}
    +	}
    +
    +	@Override
    +	protected Scan getScanner() {
    +		// TODO : Pass 'rowkey'. For this we need FilterableTableSource
    +		Scan scan = new Scan();
    +		Map<String, List<Pair>> familyMap = schema.getFamilyMap();
    +		for(String family : familyMap.keySet()) {
    +			// select only the fields in the 'selectedFields'
    +			List<Pair> colDetails = familyMap.get(family);
    +			for(Pair<String, TypeInformation<?>> pair : colDetails) {
    +				scan.addColumn(Bytes.toBytes(family), Bytes.toBytes(pair.getFirst()));
    +			}
    +		}
    +		return scan;
    +	}
    +
    +	@Override
    +	public String getTableName() {
    +		return tableName;
    +	}
    +
    +	@Override
    +	protected Row mapResultToTuple(Result res) {
    +		List<Object> values = new ArrayList<Object>();
    +		int i = 0;
    +		Map<String, List<Pair>> familyMap = schema.getFamilyMap();
    +		Row[] rows = new Row[familyMap.size()];
    +		for(String family : familyMap.keySet()) {
    +			List<Pair> colDetails = familyMap.get(family);
    +			for(Pair<String, TypeInformation<?>> pair : colDetails) {
    +				byte[] value = res.getValue(Bytes.toBytes(family), Bytes.toBytes(pair.getFirst()));
    +				if(value != null) {
    +					values.add(schema.deserialize(value, pair.getSecond()));
    +				} else {
    +					values.add(schema.deserializeNull(pair.getSecond()));
    --- End diff --
    
    >But one main concern from my side is how to present the 'NULL' means we specify a column with a type but there is no data for that column. For now I have handled by returning the Int, Float, Long - Min_values. But that may not be right I believe. Feedback and suggestions welcome.
    
    I think we can return `null` directly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Am not sure how we will manage this. But one thing that can be done is that if invalid family name is added to the scan - HBase internally throws FamilyNotFoundException - so that we can track and report back. But am not sure how to track the schema. So every time he wants to scan two families - the user has to call HBaseTableSchema#addColumns() twice (with the required qualifiers). 
    @tonycox 
    Can you help me understand - when you say ' so we can add more maps that generate after familyMap has set'?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Thanks @tonycox .Yes I am fine with it. I can try. Anyway I t hink there is more to do from my side. Now am not sure how to register the table with the valid family name and column name. It is only registering with the table and it is not resolving '.select("f1:q1, f1:q2, f1:q3");'.
    I am able to run the test case only now and so finding these. I will wait for comments and then go on with the updation of the PR.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Thanks to @tonycox  for helping me in reproducing this error. Changing to JDK 7 creates this issue and it creates due to permGen space running out of memory. I don't have a soln for this. It runs with JDK 8 with no hassles. Any inputs here?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @ramkrish86 Try to add 
    ```
    -XX:+HeapDumpOnOutOfMemoryError 
    -XX:HeapDumpPath="/tmp"
    ``` 
    as parameter of JVM


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r96783534
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,117 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.Test;
    +import scala.tools.cmd.gen.AnyVals;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +
    +	public static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	public static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	public static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	public static final byte[] F_1 = Bytes.toBytes("f1");
    +	public static final byte[] Q_1 = Bytes.toBytes("q1");
    +	public static final byte[] Q_2 = Bytes.toBytes("q2");
    +	public static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@Test
    +	public void testHBaseTableSource() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test");
    +		createTable(tableName, F_1, new byte[1][]);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_2, Bytes.toBytes(19991l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_2, Bytes.toBytes(19992l));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_2, Bytes.toBytes(19993l));
    --- End diff --
    
    Ya.  Good catch. Will change it. I was more focussed on the impl because if you see the test case itself was not asserting any. I need to add all those stuff. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r96790253
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
    +import org.apache.flink.api.common.io.RichInputFormat;
    +import org.apache.flink.api.common.io.statistics.BaseStatistics;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.io.InputSplitAssigner;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.client.Table;
    +import org.apache.hadoop.hbase.client.ClusterConnection;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.ResultScanner;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.HRegionLocator;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends RichInputFormat<Row, TableInputSplit> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private TypeInformation[] fieldTypeInfos;
    +	private String[] fieldNames;
    +	private transient Table table;
    +	private transient Scan scan;
    +	private transient Connection conn;
    +	private ResultScanner resultScanner = null;
    +
    +	private byte[] lastRow;
    +	private int scannedRows;
    +	private boolean endReached = false;
    +	private org.apache.hadoop.conf.Configuration conf;
    +	private static final String COLON = ":";
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, String[] fieldNames, TypeInformation[] fieldTypeInfos) {
    +		this.conf = conf;
    +		this.tableName = tableName;
    +		this.fieldNames = fieldNames;
    +		this.fieldTypeInfos = fieldTypeInfos;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = createScanner();
    +		}
    +	}
    +
    +	private Scan createScanner() {
    +		Scan scan = new Scan();
    +		for(String field : fieldNames) {
    +			// select only the fields in the 'selectedFields'
    +			String[] famCol = field.split(COLON);
    +			scan.addColumn(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +		}
    +		return scan;
    +	}
    +
    +	private void connectToTable() {
    +		//use files found in the classpath
    +		if(this.conf == null) {
    +			this.conf = HBaseConfiguration.create();
    +		}
    +		try {
    +			conn = ConnectionFactory.createConnection(this.conf);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while creating connection to hbase cluster", ioe);
    +			return;
    +		}
    +		try {
    +			table = conn.getTable(TableName.valueOf(tableName));
    +		} catch(TableNotFoundException tnfe) {
    +			LOG.error("The table " + tableName + " not found ", tnfe);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while connecting to the table "+tableName+ " ", ioe);
    +		}
    +	}
    +
    +	@Override
    +	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
    +		return null;
    +	}
    +
    +	@Override
    +	public TableInputSplit[] createInputSplits(final int minNumSplits) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +
    +		//Gets the starting and ending row keys for every region in the currently open table
    +		HRegionLocator regionLocator = new HRegionLocator(table.getName(), (ClusterConnection) conn);
    +		final Pair<byte[][], byte[][]> keys = regionLocator.getStartEndKeys();
    +		if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
    +			throw new IOException("Expecting at least one region.");
    +		}
    +		final byte[] startRow = scan.getStartRow();
    +		final byte[] stopRow = scan.getStopRow();
    +		final boolean scanWithNoLowerBound = startRow.length == 0;
    +		final boolean scanWithNoUpperBound = stopRow.length == 0;
    +
    +		final List<TableInputSplit> splits = new ArrayList<TableInputSplit>(minNumSplits);
    +		for (int i = 0; i < keys.getFirst().length; i++) {
    +			final byte[] startKey = keys.getFirst()[i];
    +			final byte[] endKey = keys.getSecond()[i];
    +			final String regionLocation = regionLocator.getRegionLocation(startKey, false).getHostnamePort();
    +			//Test if the given region is to be included in the InputSplit while splitting the regions of a table
    +			if (!includeRegionInSplit(startKey, endKey)) {
    +				continue;
    +			}
    +			//Finds the region on which the given row is being served
    +			final String[] hosts = new String[]{regionLocation};
    +
    +			// determine if regions contains keys used by the scan
    +			boolean isLastRegion = endKey.length == 0;
    +			if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0) &&
    +				(scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) {
    +
    +				final byte[] splitStart = scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0 ? startKey : startRow;
    +				final byte[] splitStop = (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0)
    +					&& !isLastRegion ? endKey : stopRow;
    +				int id = splits.size();
    +				final TableInputSplit split = new TableInputSplit(id, hosts, table.getName().getName(), splitStart, splitStop);
    +				splits.add(split);
    +			}
    +		}
    +		LOG.info("Created " + splits.size() + " splits");
    +		for (TableInputSplit split : splits) {
    +			logSplitInfo("created", split);
    +		}
    +		return splits.toArray(new TableInputSplit[0]);
    +	}
    +
    +	protected boolean includeRegionInSplit(final byte[] startKey, final byte[] endKey) {
    +		return true;
    +	}
    +
    +	@Override
    +	public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
    +		return new LocatableInputSplitAssigner(inputSplits);
    +	}
    +
    +	@Override
    +	public void open(TableInputSplit split) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +		if (split == null) {
    +			throw new IOException("Input split is null!");
    +		}
    +
    +		logSplitInfo("opening", split);
    +		// set the start row and stop row from the splits
    +		scan.setStartRow(split.getStartRow());
    +		lastRow = split.getEndRow();
    +		scan.setStopRow(lastRow);
    +
    +		resultScanner = table.getScanner(scan);
    +		endReached = false;
    +		scannedRows = 0;
    +	}
    +
    +	private void logSplitInfo(String action, TableInputSplit split) {
    +		int splitId = split.getSplitNumber();
    +		String splitStart = Bytes.toString(split.getStartRow());
    +		String splitEnd = Bytes.toString(split.getEndRow());
    +		String splitStartKey = splitStart.isEmpty() ? "-" : splitStart;
    +		String splitStopKey = splitEnd.isEmpty() ? "-" : splitEnd;
    +		String[] hostnames = split.getHostnames();
    +		LOG.info("{} split (this={})[{}|{}|{}|{}]", action, this, splitId, hostnames, splitStartKey, splitStopKey);
    +	}
    +
    +	@Override
    +	public boolean reachedEnd() throws IOException {
    +		return endReached;
    +	}
    +
    +	@Override
    +	public Row nextRecord(Row reuse) throws IOException {
    +		if (resultScanner == null) {
    +			throw new IOException("No table result scanner provided!");
    +		}
    +		try {
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		} catch (Exception e) {
    +			resultScanner.close();
    +			//workaround for timeout on scan
    +			LOG.warn("Error after scan of " + scannedRows + " rows. Retry with a new scanner...", e);
    +			scan.setStartRow(lastRow);
    +			resultScanner = table.getScanner(scan);
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		}
    +		endReached = true;
    +		return null;
    +	}
    +
    +	private Row mapResultToRow(Result res) {
    +		Object[] values = new Object[fieldNames.length];
    +		int i = 0;
    +		for(String field : fieldNames) {
    +			String[] famCol = field.split(COLON);
    +			byte[] value = res.getValue(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +			TypeInformation typeInfo = fieldTypeInfos[i];
    +			if(typeInfo.isBasicType()) {
    +				if(typeInfo.getTypeClass() ==  Integer.class) {
    +					values[i] = Bytes.toInt(value);
    +				} else if(typeInfo.getTypeClass() == Short.class) {
    +					values[i] = Bytes.toShort(value);
    +				} else if(typeInfo.getTypeClass() == Float.class) {
    +					values[i] = Bytes.toFloat(value);
    +				} else if(typeInfo.getTypeClass() == Long.class) {
    +					values[i] = Bytes.toLong(value);
    +				} else if(typeInfo.getTypeClass() == String.class) {
    +					values[i] = Bytes.toString(value);
    +				} else if(typeInfo.getTypeClass() == Byte.class) {
    +					values[i] = value[0];
    +				} else if(typeInfo.getTypeClass() == Boolean.class) {
    +					values[i] = Bytes.toBoolean(value);
    +				} else if(typeInfo.getTypeClass() == Double.class) {
    +					values[i] = Bytes.toDouble(value);
    +				} else if(typeInfo.getTypeClass() == BigInteger.class) {
    +					values[i] = new BigInteger(value);
    +				} else if(typeInfo.getTypeClass() == BigDecimal.class) {
    +					values[i] = Bytes.toBigDecimal(value);
    +				} else if(typeInfo.getTypeClass() == Date.class) {
    +					values[i] = new Date(Bytes.toLong(value));
    +				}
    +			} else {
    +				// TODO for other types??
    --- End diff --
    
    {code}
    testHBaseTableSource(org.apache.flink.addons.hbase.example.HBaseTableSourceITCase)  Time elapsed: 2.297 sec  <<< ERROR!
    org.apache.flink.table.api.ValidationException: Cannot resolve [q1] given input [f0, f1, f2].
            at org.apache.flink.addons.hbase.example.HBaseTableSourceITCase.testHBaseTableSource(HBaseTableSourceITCase.java:113)
    {code}
    The error says field input [f0, f1, f2]. Not sure how it got picked up.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97709482
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair>> familyMap =
    +		new HashMap<String, List<Pair>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class
    +	};
    +	private static byte[] EMPTY_BYTE_ARRAY = new byte[0];
    +	public void addColumns(String family, String qualifier, TypeInformation<?> type) {
    +		Preconditions.checkNotNull(family, "family name");
    +		Preconditions.checkNotNull(family, "qualifier name");
    +		Preconditions.checkNotNull(type, "type name");
    +		List<Pair> list = this.familyMap.get(family);
    +		if (list == null) {
    +			list = new ArrayList<Pair>();
    +		}
    +		boolean found = false;
    +		for(Class classType : CLASS_TYPES) {
    +			if(classType == type.getTypeClass()) {
    +				found = true;
    +				break;
    +			}
    +		}
    +		if(!found) {
    +			// by default it will be byte[] type only
    +			type = BasicArrayTypeInfo.BYTE_ARRAY_TYPE_INFO;
    --- End diff --
    
    Ok. Got it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Thanks to all @fhueske , @tonycox  and @wuchong for helping in getting this in.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Updated the code with the comments and have pushed again. I think I have addressed all the comments here. Feedback/comments welcome. I also found that it is better to use the TableInputSplit to specify the start and end row so that the scan is anyway restricted to the given range.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98487401
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,140 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.sql.Time;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.sql.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    --- End diff --
    
    > I think PushProjectIntoBatchTableSourceScanRule is not good enough for nested data types. but we can project at least family columns now.
    
    So you mean we will project the column families. One thing to remember is that though we have two families - Eduction and Department. When we retrieve Department would come first and then Education. So what matters is how we retrieve the result. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    `02/02/2017 06:19:20	DataSink (collect())(6/32) switched to SCHEDULED 
    02/02/2017 06:19:20	DataSink (collect())(2/32) switched to SCHEDULED 
    02/02/2017 06:19:20	DataSink (collect())(8/32) switched to SCHEDULED 
    02/02/2017 06:19:20	DataSink (collect())(6/32) switched to DEPLOYING 
    02/02/2017 06:19:20	DataSink (collect())(8/32) switched to DEPLOYING 
    02/02/2017 06:19:20	DataSink (collect())(2/32) switched to DEPLOYING 
    02/02/2017 06:19:25	DataSink (collect())(1/32) switched to SCHEDULED 
    02/02/2017 06:19:25	DataSink (collect())(1/32) switched to DEPLOYING 
    Exception: java.lang.OutOfMemoryError thrown from the UncaughtExceptionHandler in thread "JvmPauseMonitor"
    Exception: java.lang.OutOfMemoryError thrown from the UncaughtExceptionHandler in thread "hconnection-0x2247c79b-shared--pool20-t1"
    Exception: java.lang.OutOfMemoryError thrown from the UncaughtExceptionHandler in thread "JvmPauseMonitor"
    Exception: java.lang.OutOfMemoryError thrown from the UncaughtExceptionHandler in thread "flink-akka.actor.default-dispatcher-3"
    Exception: java.lang.OutOfMemoryError thrown from the UncaughtExceptionHandler in thread "IPC Server handler 1 on 34267"
    Exception: java.lang.OutOfMemoryError thrown from the UncaughtExceptionHandler in thread "CHAIN DataSource (localhost:53456)"
    Exception: java.lang.OutOfMemoryError thrown from the UncaughtExceptionHandler in thread "CHAIN DataSource (localhost:53456)"
    Exception: java.lang.OutOfMemoryError thrown from the UncaughtExceptionHandler in thread "CHAIN DataSource (localhost:53456)"
    Exception: java.lang.OutOfMemoryError thrown from the UncaughtExceptionHandler in thread "org.apache.hadoop.hdfs.PeerCache@28360f4a"`
    
    The same tests run cleanly in my linux box. Is there any place I can download the logs to see what could be the issue?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    @fhueske - A gentle reminder !!!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    And could you extend with `StreamTableSource` also ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    Sorry for the late response. 
    
    Regarding to the `HBaseTableSchema`, I agree with that to move the `addColumn(...)` method into `HBaseTableSource`. 
    
    Regarding to the nested vs flat schema, I prefer the nested schema. It is more intuitive to use.
    As for the nested schema doesn't support to push projections down, I think we should extend `ProjectableTableSource` to support push projections down to a composite type. We can keep the interface unchanged, i.e. `def projectFields(fields: Array[Int]): ProjectableTableSource[T]`. But the index of `fields` should be the flat index. We can use the flat field indexes to do projection pushdown even if it is a nested schema.
    
    For example, a table source with schema `a: Int, b: Row<b1: String, b2: Int>, c: Boolean`, the flat indexes of `a, b.b1, b.b2, c` are `0, 1, 2, 3`. So a project `SELECT b.b1, c FROM T` will result a `fields` `Array(1,3)`.
    
    What do you think ? 
    
    
    For me the biggest drawback of a nested schema is the lacking support to push projections down. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    As Jark Wu said in [jira](https://issues.apache.org/jira/browse/FLINK-5554) 
    
    > I think the HBaseTableSource should return a composite type (with column family and qualifier), and we can get columns by composite type accessing.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    And one more thing is that other than the BasicTypeInfo what other types should we support. I was not sure on that so added a TODO there.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97934428
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.util.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    +		new HashMap<String, List<Pair<String, TypeInformation<?>>>>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    --- End diff --
    
    Ok. Makes sense.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/3149
  
    The points I raised effect the API and different people tend to have different opinions about APIs ;-).
    
    I suggested to use a flat schema with that names columns `columnFamily$qualifier`, i.e., no nesting but composing the column name from `colFamily` and `qualifier` and separating them by `$`. Internally we can and should still use the `family`-`qualifier` but just map everything to a flat schema. The question is whether this would that make the `HBaseTableSource` harder to use. I don't think the column access (`family.qualifier` vs. `family$qualifier`) would be much harder, but acting with families that have lots of columns would be more cumbersome, because each column would be a top-level column and would need to be explicitly selected in a `SELECT` clause. In the nested case, all columns of a family are conveniently grouped together.
    
    Regarding the `HBaseTableSchema`, we could also use it only internally and not expose it to the user. The `HBaseTableSource` would have a method `addColumn()` and forward the calls to its internal `HBaseSchema`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by tonycox <gi...@git.apache.org>.
Github user tonycox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98434792
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSchema.java ---
    @@ -0,0 +1,140 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.util.Preconditions;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.io.Serializable;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.sql.Time;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.ArrayList;
    +import java.sql.Date;
    +
    +/**
    + * Helps to specify an HBase Table's schema
    + */
    +public class HBaseTableSchema implements Serializable {
    +
    +	// A Map with key as column family.
    +	private final Map<String, List<Pair<String, TypeInformation<?>>>> familyMap =
    +		new HashMap<>();
    +
    +	// Allowed types. This may change.
    +	// TODO : Check if the Date type should be the one in java.util or the one in java.sql
    +	private static Class[] CLASS_TYPES = {
    +		Integer.class, Short.class, Float.class, Long.class, String.class, Byte.class, Boolean.class, Double.class, BigInteger.class, BigDecimal.class, Date.class, Time.class, byte[].class
    --- End diff --
    
    why is this static class?
    what do you think to use an immutable list? for example `com.google.common.collect.ImmutableCollection`
    ```java
    private static ImmutableCollection<Class<?>> CLASS_TYPES = ImmutableList.<Class<?>>of(...
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by ramkrish86 <gi...@git.apache.org>.
Github user ramkrish86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r97015379
  
    --- Diff: flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSourceInputFormat.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.flink.addons.hbase;
    +
    +import org.apache.flink.api.common.io.InputFormat;
    +import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
    +import org.apache.flink.api.common.io.RichInputFormat;
    +import org.apache.flink.api.common.io.statistics.BaseStatistics;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.io.InputSplitAssigner;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.HBaseConfiguration;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.TableNotFoundException;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.client.Table;
    +import org.apache.hadoop.hbase.client.ClusterConnection;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.ResultScanner;
    +import org.apache.hadoop.hbase.client.ConnectionFactory;
    +import org.apache.hadoop.hbase.client.Connection;
    +import org.apache.hadoop.hbase.client.HRegionLocator;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.hadoop.hbase.util.Pair;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +/**
    + * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row}
    + */
    +public class HBaseTableSourceInputFormat extends RichInputFormat<Row, TableInputSplit> implements ResultTypeQueryable<Row> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(HBaseTableSourceInputFormat.class);
    +	private String tableName;
    +	private TypeInformation[] fieldTypeInfos;
    +	private String[] fieldNames;
    +	private transient Table table;
    +	private transient Scan scan;
    +	private transient Connection conn;
    +	private ResultScanner resultScanner = null;
    +
    +	private byte[] lastRow;
    +	private int scannedRows;
    +	private boolean endReached = false;
    +	private org.apache.hadoop.conf.Configuration conf;
    +	private static final String COLON = ":";
    +
    +	public HBaseTableSourceInputFormat(org.apache.hadoop.conf.Configuration conf, String tableName, String[] fieldNames, TypeInformation[] fieldTypeInfos) {
    +		this.conf = conf;
    +		this.tableName = tableName;
    +		this.fieldNames = fieldNames;
    +		this.fieldTypeInfos = fieldTypeInfos;
    +	}
    +
    +	@Override
    +	public void configure(Configuration parameters) {
    +		LOG.info("Initializing HBaseConfiguration");
    +		connectToTable();
    +		if(table != null) {
    +			scan = createScanner();
    +		}
    +	}
    +
    +	private Scan createScanner() {
    +		Scan scan = new Scan();
    +		for(String field : fieldNames) {
    +			// select only the fields in the 'selectedFields'
    +			String[] famCol = field.split(COLON);
    +			scan.addColumn(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +		}
    +		return scan;
    +	}
    +
    +	private void connectToTable() {
    +		//use files found in the classpath
    +		if(this.conf == null) {
    +			this.conf = HBaseConfiguration.create();
    +		}
    +		try {
    +			conn = ConnectionFactory.createConnection(this.conf);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while creating connection to hbase cluster", ioe);
    +			return;
    +		}
    +		try {
    +			table = conn.getTable(TableName.valueOf(tableName));
    +		} catch(TableNotFoundException tnfe) {
    +			LOG.error("The table " + tableName + " not found ", tnfe);
    +		} catch(IOException ioe) {
    +			LOG.error("Exception while connecting to the table "+tableName+ " ", ioe);
    +		}
    +	}
    +
    +	@Override
    +	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
    +		return null;
    +	}
    +
    +	@Override
    +	public TableInputSplit[] createInputSplits(final int minNumSplits) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +
    +		//Gets the starting and ending row keys for every region in the currently open table
    +		HRegionLocator regionLocator = new HRegionLocator(table.getName(), (ClusterConnection) conn);
    +		final Pair<byte[][], byte[][]> keys = regionLocator.getStartEndKeys();
    +		if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
    +			throw new IOException("Expecting at least one region.");
    +		}
    +		final byte[] startRow = scan.getStartRow();
    +		final byte[] stopRow = scan.getStopRow();
    +		final boolean scanWithNoLowerBound = startRow.length == 0;
    +		final boolean scanWithNoUpperBound = stopRow.length == 0;
    +
    +		final List<TableInputSplit> splits = new ArrayList<TableInputSplit>(minNumSplits);
    +		for (int i = 0; i < keys.getFirst().length; i++) {
    +			final byte[] startKey = keys.getFirst()[i];
    +			final byte[] endKey = keys.getSecond()[i];
    +			final String regionLocation = regionLocator.getRegionLocation(startKey, false).getHostnamePort();
    +			//Test if the given region is to be included in the InputSplit while splitting the regions of a table
    +			if (!includeRegionInSplit(startKey, endKey)) {
    +				continue;
    +			}
    +			//Finds the region on which the given row is being served
    +			final String[] hosts = new String[]{regionLocation};
    +
    +			// determine if regions contains keys used by the scan
    +			boolean isLastRegion = endKey.length == 0;
    +			if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0) &&
    +				(scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) {
    +
    +				final byte[] splitStart = scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0 ? startKey : startRow;
    +				final byte[] splitStop = (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0)
    +					&& !isLastRegion ? endKey : stopRow;
    +				int id = splits.size();
    +				final TableInputSplit split = new TableInputSplit(id, hosts, table.getName().getName(), splitStart, splitStop);
    +				splits.add(split);
    +			}
    +		}
    +		LOG.info("Created " + splits.size() + " splits");
    +		for (TableInputSplit split : splits) {
    +			logSplitInfo("created", split);
    +		}
    +		return splits.toArray(new TableInputSplit[0]);
    +	}
    +
    +	protected boolean includeRegionInSplit(final byte[] startKey, final byte[] endKey) {
    +		return true;
    +	}
    +
    +	@Override
    +	public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
    +		return new LocatableInputSplitAssigner(inputSplits);
    +	}
    +
    +	@Override
    +	public void open(TableInputSplit split) throws IOException {
    +		if (table == null) {
    +			throw new IOException("The HBase table has not been opened!");
    +		}
    +		if (scan == null) {
    +			throw new IOException("getScanner returned null");
    +		}
    +		if (split == null) {
    +			throw new IOException("Input split is null!");
    +		}
    +
    +		logSplitInfo("opening", split);
    +		// set the start row and stop row from the splits
    +		scan.setStartRow(split.getStartRow());
    +		lastRow = split.getEndRow();
    +		scan.setStopRow(lastRow);
    +
    +		resultScanner = table.getScanner(scan);
    +		endReached = false;
    +		scannedRows = 0;
    +	}
    +
    +	private void logSplitInfo(String action, TableInputSplit split) {
    +		int splitId = split.getSplitNumber();
    +		String splitStart = Bytes.toString(split.getStartRow());
    +		String splitEnd = Bytes.toString(split.getEndRow());
    +		String splitStartKey = splitStart.isEmpty() ? "-" : splitStart;
    +		String splitStopKey = splitEnd.isEmpty() ? "-" : splitEnd;
    +		String[] hostnames = split.getHostnames();
    +		LOG.info("{} split (this={})[{}|{}|{}|{}]", action, this, splitId, hostnames, splitStartKey, splitStopKey);
    +	}
    +
    +	@Override
    +	public boolean reachedEnd() throws IOException {
    +		return endReached;
    +	}
    +
    +	@Override
    +	public Row nextRecord(Row reuse) throws IOException {
    +		if (resultScanner == null) {
    +			throw new IOException("No table result scanner provided!");
    +		}
    +		try {
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		} catch (Exception e) {
    +			resultScanner.close();
    +			//workaround for timeout on scan
    +			LOG.warn("Error after scan of " + scannedRows + " rows. Retry with a new scanner...", e);
    +			scan.setStartRow(lastRow);
    +			resultScanner = table.getScanner(scan);
    +			Result res = resultScanner.next();
    +			if (res != null) {
    +				scannedRows++;
    +				lastRow = res.getRow();
    +				return mapResultToRow(res);
    +			}
    +		}
    +		endReached = true;
    +		return null;
    +	}
    +
    +	private Row mapResultToRow(Result res) {
    +		Object[] values = new Object[fieldNames.length];
    +		int i = 0;
    +		for(String field : fieldNames) {
    +			String[] famCol = field.split(COLON);
    +			byte[] value = res.getValue(Bytes.toBytes(famCol[0]), Bytes.toBytes(famCol[1]));
    +			TypeInformation typeInfo = fieldTypeInfos[i];
    +			if(typeInfo.isBasicType()) {
    +				if(typeInfo.getTypeClass() ==  Integer.class) {
    +					values[i] = Bytes.toInt(value);
    +				} else if(typeInfo.getTypeClass() == Short.class) {
    +					values[i] = Bytes.toShort(value);
    +				} else if(typeInfo.getTypeClass() == Float.class) {
    +					values[i] = Bytes.toFloat(value);
    +				} else if(typeInfo.getTypeClass() == Long.class) {
    +					values[i] = Bytes.toLong(value);
    +				} else if(typeInfo.getTypeClass() == String.class) {
    +					values[i] = Bytes.toString(value);
    +				} else if(typeInfo.getTypeClass() == Byte.class) {
    +					values[i] = value[0];
    +				} else if(typeInfo.getTypeClass() == Boolean.class) {
    +					values[i] = Bytes.toBoolean(value);
    +				} else if(typeInfo.getTypeClass() == Double.class) {
    +					values[i] = Bytes.toDouble(value);
    +				} else if(typeInfo.getTypeClass() == BigInteger.class) {
    +					values[i] = new BigInteger(value);
    +				} else if(typeInfo.getTypeClass() == BigDecimal.class) {
    +					values[i] = Bytes.toBigDecimal(value);
    +				} else if(typeInfo.getTypeClass() == Date.class) {
    +					values[i] = new Date(Bytes.toLong(value));
    +				}
    +			} else {
    +				// TODO for other types??
    --- End diff --
    
    HBaseTableSchema - So can we add the startKey and endKey also into this then?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3149: FLINK-2168 Add HBaseTableSource

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3149#discussion_r98805834
  
    --- Diff: flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseTableSourceITCase.java ---
    @@ -0,0 +1,196 @@
    +/*
    + * Copyright The Apache Software Foundation
    + *
    + * 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.flink.addons.hbase.example;
    +
    +import org.apache.flink.addons.hbase.HBaseTableSchema;
    +import org.apache.flink.addons.hbase.HBaseTableSource;
    +import org.apache.flink.addons.hbase.HBaseTestingClusterAutostarter;
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.table.api.Table;
    +import org.apache.flink.table.api.TableConfig;
    +import org.apache.flink.table.api.TableEnvironment;
    +import org.apache.flink.table.api.java.BatchTableEnvironment;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.test.util.TestBaseUtils;
    +import org.apache.flink.types.Row;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HBaseTableSourceITCase extends HBaseTestingClusterAutostarter {
    +
    +	private static final byte[] ROW_1 = Bytes.toBytes("row1");
    +	private static final byte[] ROW_2 = Bytes.toBytes("row2");
    +	private static final byte[] ROW_3 = Bytes.toBytes("row3");
    +	private static final byte[] F_1 = Bytes.toBytes("f1");
    +	private static final byte[] F_2 = Bytes.toBytes("f2");
    +	private static final byte[] Q_1 = Bytes.toBytes("q1");
    +	private static final byte[] Q_2 = Bytes.toBytes("q2");
    +	private static final byte[] Q_3 = Bytes.toBytes("q3");
    +
    +	@BeforeClass
    +	public static void activateHBaseCluster(){
    +		registerHBaseMiniClusterInClasspath();
    +	}
    +
    +	@Test
    +	public void testHBaseTableSourceWithSingleColumnFamily() throws Exception {
    +		// create a table with single region
    +		TableName tableName = TableName.valueOf("test");
    +		// no split keys
    +		byte[][] famNames = new byte[1][];
    +		famNames[0] = F_1;
    +		createTable(tableName, famNames, null);
    +		// get the htable instance
    +		HTable table = openTable(tableName);
    +		List<Put> puts = new ArrayList<Put>();
    +		// add some data
    +		Put put = new Put(ROW_1);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(100));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19991L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_2);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(101));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue1"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19992L));
    +		puts.add(put);
    +
    +		put = new Put(ROW_3);
    +		// add 3 qualifiers per row
    +		//1st qual is integer
    +		put.addColumn(F_1, Q_1, Bytes.toBytes(102));
    +		//2nd qual is String
    +		put.addColumn(F_1, Q_2, Bytes.toBytes("strvalue2"));
    +		// 3rd qual is long
    +		put.addColumn(F_1, Q_3, Bytes.toBytes(19993L));
    +		puts.add(put);
    +		// add the mutations to the table
    +		table.put(puts);
    +		table.close();
    +		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    --- End diff --
    
    add a new line and a comment that the preparation is done.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---