You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metamodel.apache.org by ka...@apache.org on 2018/06/21 05:50:52 UTC

[01/31] metamodel git commit: Notice with the builders that I tried to follow the setup used in other projects, but this often didn't work for HBase's column-families. The FindQualifiersDriver works, but only from command line within a VM which runs Hado

Repository: metamodel
Updated Branches:
  refs/heads/master a868f71c0 -> 1d67d3086


Notice with the builders that I tried to follow the setup used in other projects, but this often didn't work for HBase's column-families.
The FindQualifiersDriver works, but only from command line within a VM which runs Hadoop + HBase. It shows HBase working in combination with MapReduce. The class will be improved so it's method can be called by other classes.
All other classes have been tested by connecting it to our DataCleaner. I'll make more Unit Tests next.
hbase/pom.xml changes: With 'hbase-client' I needed to remove 2 exclusion. I've added 'hbase-server' dependency.


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/6ad602bb
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/6ad602bb
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/6ad602bb

Branch: refs/heads/master
Commit: 6ad602bb9af70ed7214b6d7e4f00cd1d9a037f71
Parents: fe02a59
Author: HUMANINFERENCE\g.dellemann <g....@quadient.com>
Authored: Fri Apr 20 16:36:27 2018 +0200
Committer: HUMANINFERENCE\g.dellemann <g....@quadient.com>
Committed: Fri Apr 20 16:36:27 2018 +0200

----------------------------------------------------------------------
 .../metamodel/AbstractUpdateCallback.java       |   4 +-
 .../metamodel/data/AbstractRowBuilder.java      |  33 ++++-
 .../insert/AbstractRowInsertionBuilder.java     |  10 +-
 hbase/pom.xml                                   | 111 +++++++++++++++
 .../org/apache/metamodel/hbase/HBaseColumn.java |  50 +++++++
 .../hbase/HBaseCreateTableBuilder.java          |  71 ++++++++++
 .../metamodel/hbase/HBaseDataContext.java       |  44 +++---
 .../apache/metamodel/hbase/HBaseFamilyMap.java  |   5 +-
 .../org/apache/metamodel/hbase/HBaseRow.java    |   6 +-
 .../hbase/HBaseRowInsertionBuilder.java         |  76 ++++++++++
 .../metamodel/hbase/HBaseTableDropBuilder.java  |  37 +++++
 .../metamodel/hbase/HBaseUpdateCallback.java    | 134 ++++++++++++++++++
 .../org/apache/metamodel/hbase/HBaseWriter.java | 138 +++++++++++++++++++
 .../hbase/qualifiers/FindQualifiersDriver.java  | 115 ++++++++++++++++
 14 files changed, 800 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/core/src/main/java/org/apache/metamodel/AbstractUpdateCallback.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/metamodel/AbstractUpdateCallback.java b/core/src/main/java/org/apache/metamodel/AbstractUpdateCallback.java
index 4ff7279..b41b481 100644
--- a/core/src/main/java/org/apache/metamodel/AbstractUpdateCallback.java
+++ b/core/src/main/java/org/apache/metamodel/AbstractUpdateCallback.java
@@ -117,7 +117,7 @@ public abstract class AbstractUpdateCallback implements UpdateCallback {
         return update(getTable(tableName));
     }
 
-    private Table getTable(String tableName) {
+    protected Table getTable(String tableName) {
         Table table = getDataContext().getTableByQualifiedLabel(tableName);
         if (table == null) {
             throw new IllegalArgumentException("No such table: " + tableName);
@@ -159,7 +159,7 @@ public abstract class AbstractUpdateCallback implements UpdateCallback {
             UnsupportedOperationException {
         return new DeleteAndInsertBuilder(this, table);
     }
-    
+
     public UpdateSummary getUpdateSummary() {
         return DefaultUpdateSummary.unknownUpdates();
     }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/core/src/main/java/org/apache/metamodel/data/AbstractRowBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/metamodel/data/AbstractRowBuilder.java b/core/src/main/java/org/apache/metamodel/data/AbstractRowBuilder.java
index 144017c..4fdbc59 100644
--- a/core/src/main/java/org/apache/metamodel/data/AbstractRowBuilder.java
+++ b/core/src/main/java/org/apache/metamodel/data/AbstractRowBuilder.java
@@ -40,6 +40,10 @@ public abstract class AbstractRowBuilder<RB extends RowBuilder<?>> implements Ro
         this(table.getColumns());
     }
 
+    public AbstractRowBuilder(Table table, int numberOfValues) {
+        this(table.getColumns(), numberOfValues);
+    }
+
     public AbstractRowBuilder(List<Column> columns) {
         _columns = columns.toArray(new Column[columns.size()]);
         _explicitNulls = new boolean[_columns.length];
@@ -47,6 +51,14 @@ public abstract class AbstractRowBuilder<RB extends RowBuilder<?>> implements Ro
         _styles = new Style[_columns.length];
     }
 
+    public AbstractRowBuilder(List<Column> columns, int numberOfValues) {
+        _columns = columns.toArray(new Column[columns.size()]);
+        _explicitNulls = new boolean[numberOfValues];
+        _values = new Object[numberOfValues];
+        _styles = new Style[numberOfValues];
+        setColumns(columns);
+    }
+
     /**
      * Gets a boolean array indicating if any of the values have been explicitly
      * set to null (as opposed to just not set)
@@ -71,7 +83,8 @@ public abstract class AbstractRowBuilder<RB extends RowBuilder<?>> implements Ro
 
     @Override
     public final Row toRow() {
-        return new DefaultRow(new SimpleDataSetHeader(Arrays.stream(_columns).map(SelectItem::new).collect(Collectors.toList())), _values);
+        return new DefaultRow(new SimpleDataSetHeader(Arrays.stream(_columns).map(SelectItem::new).collect(Collectors
+                .toList())), _values);
     }
 
     @Override
@@ -146,4 +159,22 @@ public abstract class AbstractRowBuilder<RB extends RowBuilder<?>> implements Ro
         }
         return false;
     }
+
+    public void setColumns(List<Column> columns) {
+        if (columns.size() != _columns.length) {
+            throw new IllegalArgumentException("The amount of columns don't match");
+        }
+        for (int i = 0; i < _columns.length; i++) {
+            _columns[i] = columns.get(i);
+        }
+    }
+
+    public void setValues(Object[] values) {
+        if (values.length != _values.length) {
+            throw new IllegalArgumentException("The amount of values don't match");
+        }
+        for (int i = 0; i < values.length; i++) {
+            _values[i] = values[i];
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java b/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
index 58c0da8..185f942 100644
--- a/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
+++ b/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
@@ -18,6 +18,8 @@
  */
 package org.apache.metamodel.insert;
 
+import java.util.List;
+
 import org.apache.metamodel.UpdateCallback;
 import org.apache.metamodel.data.AbstractRowBuilder;
 import org.apache.metamodel.data.Row;
@@ -25,8 +27,6 @@ import org.apache.metamodel.query.SelectItem;
 import org.apache.metamodel.schema.Column;
 import org.apache.metamodel.schema.Table;
 
-import java.util.List;
-
 /**
  * Abstract implementation of the {@link RowInsertionBuilder} interface,
  * provided as a convenience to {@link RowInsertable} implementations. Handles
@@ -44,6 +44,12 @@ public abstract class AbstractRowInsertionBuilder<U extends UpdateCallback> exte
         _table = table;
     }
 
+    public AbstractRowInsertionBuilder(U updateCallback, Table table, int numberOfValues) {
+        super(table, numberOfValues);
+        _updateCallback = updateCallback;
+        _table = table;
+    }
+
     @Override
     public Table getTable() {
         return _table;

http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/hbase/pom.xml
----------------------------------------------------------------------
diff --git a/hbase/pom.xml b/hbase/pom.xml
index 5a0cd99..aed4f63 100644
--- a/hbase/pom.xml
+++ b/hbase/pom.xml
@@ -118,6 +118,113 @@
 					<groupId>commons-httpclient</groupId>
 					<artifactId>commons-httpclient</artifactId>
 				</exclusion>
+				<!-- <exclusion>
+					<groupId>org.codehaus.jackson</groupId>
+					<artifactId>jackson-mapper-asl</artifactId>
+				</exclusion> -->
+				<exclusion>
+					<groupId>org.codehaus.jackson</groupId>
+					<artifactId>jackson-core</artifactId>
+				</exclusion>
+				<!-- <exclusion>
+					<groupId>org.codehaus.jackson</groupId>
+					<artifactId>jackson-core-asl</artifactId>
+				</exclusion> -->
+				<exclusion>
+					<groupId>org.codehaus.jackson</groupId>
+					<artifactId>jackson-jaxrs</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.codehaus.jackson</groupId>
+					<artifactId>jackson-xc</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+				<dependency>
+			<groupId>org.apache.hbase</groupId>
+			<artifactId>hbase-server</artifactId>
+			<version>${hbase.version}</version>
+			<exclusions>
+				<exclusion>
+					<groupId>jdk.tools</groupId>
+					<artifactId>jdk.tools</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.slf4j</groupId>
+					<artifactId>slf4j-log4j12</artifactId>
+				</exclusion>
+				<exclusion>
+					<artifactId>log4j</artifactId>
+					<groupId>log4j</groupId>
+				</exclusion>
+				<exclusion>
+					<artifactId>commons-logging</artifactId>
+					<groupId>commons-logging</groupId>
+				</exclusion>
+				<exclusion>
+					<artifactId>jetty</artifactId>
+					<groupId>org.mortbay.jetty</groupId>
+				</exclusion>
+				<exclusion>
+					<artifactId>jetty-util</artifactId>
+					<groupId>org.mortbay.jetty</groupId>
+				</exclusion>
+				<exclusion>
+						<groupId>com.github.stephenc.findbugs</groupId>
+						<artifactId>findbugs-annotations</artifactId>
+					</exclusion>
+				<exclusion>
+					<groupId>tomcat</groupId>
+					<artifactId>jasper-runtime</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>tomcat</groupId>
+					<artifactId>jasper-compiler</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>javax.servlet</groupId>
+					<artifactId>servlet-api</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>google-collections</groupId>
+					<artifactId>google-collections</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>net.sourceforge.collections</groupId>
+					<artifactId>collections-generic</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.sun.jersey</groupId>
+					<artifactId>jersey-core</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.sun.jersey</groupId>
+					<artifactId>jersey-server</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jsp-2.1</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.sun.jersey</groupId>
+					<artifactId>jersey-json</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jsp-api-2.1</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>io.netty</groupId>
+					<artifactId>netty</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>io.netty</groupId>
+					<artifactId>netty-all</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>commons-httpclient</groupId>
+					<artifactId>commons-httpclient</artifactId>
+				</exclusion>
 				<exclusion>
 					<groupId>org.codehaus.jackson</groupId>
 					<artifactId>jackson-mapper-asl</artifactId>
@@ -138,6 +245,10 @@
 					<groupId>org.codehaus.jackson</groupId>
 					<artifactId>jackson-xc</artifactId>
 				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>servlet-api-2.5</artifactId>
+				</exclusion>
 			</exclusions>
 		</dependency>
 		<dependency>

http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
new file mode 100644
index 0000000..abe37d1
--- /dev/null
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
@@ -0,0 +1,50 @@
+/**
+ * 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.metamodel.hbase;
+
+public final class HBaseColumn {
+    private String columnFamily;
+    private String qualifier;
+
+    public HBaseColumn() {
+        columnFamily = "";
+        qualifier = "";
+    }
+
+    public HBaseColumn(String columnFamily, String qualifier) {
+        this.columnFamily = columnFamily;
+        this.qualifier = qualifier;
+    }
+
+    public String getColumnFamily() {
+        return columnFamily;
+    }
+
+    public void setColumnFamily(String columnFamily) {
+        this.columnFamily = columnFamily;
+    }
+
+    public String getQualifier() {
+        return qualifier;
+    }
+
+    public void setQualifier(String qualifier) {
+        this.qualifier = qualifier;
+    }
+}

http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
new file mode 100644
index 0000000..e1f1257
--- /dev/null
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
@@ -0,0 +1,71 @@
+/**
+ * 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.metamodel.hbase;
+
+import java.io.IOException;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import org.apache.metamodel.MetaModelException;
+import org.apache.metamodel.create.AbstractTableCreationBuilder;
+import org.apache.metamodel.schema.MutableSchema;
+import org.apache.metamodel.schema.MutableTable;
+import org.apache.metamodel.schema.Schema;
+import org.apache.metamodel.schema.Table;
+import org.apache.metamodel.util.SimpleTableDef;
+
+public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseUpdateCallback> {
+
+    private Set<String> columnFamilies;
+
+    public HBaseCreateTableBuilder(HBaseUpdateCallback updateCallback, Schema schema, String name,
+            HBaseColumn[] outputColumns) {
+        super(updateCallback, schema, name);
+        if (!(schema instanceof MutableSchema)) {
+            throw new IllegalArgumentException("Not a valid schema: " + schema);
+        }
+        columnFamilies = new LinkedHashSet<String>();
+        for (int i = 0; i < outputColumns.length; i++) {
+            columnFamilies.add(outputColumns[i].getColumnFamily());
+        }
+    }
+
+    @Override
+    public Table execute() throws MetaModelException {
+        final MutableTable table = getTable();
+        final SimpleTableDef emptyTableDef = new SimpleTableDef(table.getName(), columnFamilies.toArray(
+                new String[columnFamilies.size()]));
+
+        final HBaseUpdateCallback updateCallback = (HBaseUpdateCallback) getUpdateCallback();
+
+        try {
+            final HBaseWriter HbaseWriter = new HBaseWriter(HBaseDataContext.createConfig(updateCallback
+                    .getConfiguration()));
+            HbaseWriter.createTable(table.getName(), columnFamilies);
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+
+        final MutableSchema schema = (MutableSchema) table.getSchema();
+        schema.addTable(new HBaseTable(updateCallback.getDataContext(), emptyTableDef, schema,
+                HBaseConfiguration.DEFAULT_ROW_KEY_TYPE));
+        return schema.getTableByName(table.getName());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
index 46866b7..6b48723 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
@@ -19,7 +19,6 @@
 package org.apache.metamodel.hbase;
 
 import java.io.IOException;
-import java.lang.reflect.Method;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
@@ -73,6 +72,9 @@ public class HBaseDataContext extends QueryPostprocessDataContext {
         Configuration config = createConfig(configuration);
         _configuration = configuration;
         _connection = createConnection(config);
+
+        // HBaseUpdateCallback updateCallback = new HBaseUpdateCallback(this);
+        // updateCallback.writeRow();
     }
 
     /**
@@ -85,6 +87,9 @@ public class HBaseDataContext extends QueryPostprocessDataContext {
         super(false);
         _configuration = configuration;
         _connection = connection;
+
+        // HBaseUpdateCallback updateCallback = new HBaseUpdateCallback(this);
+        // updateCallback.writeRow();
     }
 
     private Connection createConnection(Configuration config) {
@@ -95,7 +100,7 @@ public class HBaseDataContext extends QueryPostprocessDataContext {
         }
     }
 
-    private Configuration createConfig(HBaseConfiguration configuration) {
+    static protected Configuration createConfig(HBaseConfiguration configuration) {
         Configuration config = org.apache.hadoop.hbase.HBaseConfiguration.create();
         config.set("hbase.zookeeper.quorum", configuration.getZookeeperHostname());
         config.set("hbase.zookeeper.property.clientPort", Integer.toString(configuration.getZookeeperPort()));
@@ -123,28 +128,28 @@ public class HBaseDataContext extends QueryPostprocessDataContext {
     }
 
     @Override
-    protected Schema getMainSchema() throws MetaModelException {
+    public Schema getMainSchema() throws MetaModelException {
         final MutableSchema schema = new MutableSchema(_configuration.getSchemaName());
 
-        try {
-            SimpleTableDef[] tableDefinitions = _configuration.getTableDefinitions();
-            if (tableDefinitions == null) {
+        SimpleTableDef[] tableDefinitions = _configuration.getTableDefinitions();
+        if (tableDefinitions == null) {
+            try {
                 final HTableDescriptor[] tables = getAdmin().listTables();
                 tableDefinitions = new SimpleTableDef[tables.length];
                 for (int i = 0; i < tables.length; i++) {
                     SimpleTableDef emptyTableDef = new SimpleTableDef(tables[i].getNameAsString(), new String[0]);
                     tableDefinitions[i] = emptyTableDef;
                 }
+            } catch (IllegalArgumentException | IOException e) {
+                throw new MetaModelException(e);
             }
+        }
 
-            for (SimpleTableDef tableDef : tableDefinitions) {
-                schema.addTable(new HBaseTable(this, tableDef, schema, _configuration.getDefaultRowKeyType()));
-            }
-
-            return schema;
-        } catch (Exception e) {
-            throw new MetaModelException(e);
+        for (SimpleTableDef tableDef : tableDefinitions) {
+            schema.addTable(new HBaseTable(this, tableDef, schema, _configuration.getDefaultRowKeyType()));
         }
+
+        return schema;
     }
 
     /**
@@ -242,17 +247,6 @@ public class HBaseDataContext extends QueryPostprocessDataContext {
     }
 
     private void setMaxRows(Scan scan, int maxRows) {
-        try {
-            // in old versions of the HBase API, the 'setMaxResultSize' method
-            // is not available
-            Method method = scan.getClass().getMethod("setMaxResultSize", long.class);
-            method.invoke(scan, (long) maxRows);
-            logger.debug("Succesfully set maxRows using Scan.setMaxResultSize({})", maxRows);
-        } catch (Exception e) {
-            logger.debug(
-                    "HBase API does not have Scan.setMaxResultSize(long) method, setting maxRows using PageFilter.", e);
-            scan.setFilter(new PageFilter(maxRows));
-        }
+        scan.setFilter(new PageFilter(maxRows));
     }
-
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java
index 38a6848..7483c63 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java
@@ -18,7 +18,6 @@
  */
 package org.apache.metamodel.hbase;
 
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Map;
 import java.util.NavigableMap;
@@ -106,9 +105,9 @@ public class HBaseFamilyMap implements Map<Object, Object> {
             if (sb.length() > 1) {
                 sb.append(',');
             }
-            sb.append(Arrays.toString(entry.getKey()));
+            sb.append(new String(entry.getKey()));
             sb.append('=');
-            sb.append(Arrays.toString(entry.getValue()));
+            sb.append(new String(entry.getValue()));
         }
         sb.append('}');
         return sb.toString();

http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java
index b091ae1..2820602 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java
@@ -59,9 +59,13 @@ final class HBaseRow extends AbstractRow implements Row {
             }
             return rowKey;
         }
-
         final int colonIndex = name.indexOf(':');
         if (colonIndex != -1) {
+            /*
+             * I think this is DEATH code! The first line of this method: implementation
+             * _header.getSelectItem(index).getColumn() will always return only a columnfamily with our current
+             * implementations.
+             */
             byte[] family = name.substring(0, colonIndex).getBytes();
             byte[] qualifier = name.substring(colonIndex + 1).getBytes();
             byte[] value = _result.getValue(family, qualifier);

http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
new file mode 100644
index 0000000..358d743
--- /dev/null
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
@@ -0,0 +1,76 @@
+/**
+ * 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.metamodel.hbase;
+
+import org.apache.metamodel.MetaModelException;
+import org.apache.metamodel.insert.AbstractRowInsertionBuilder;
+import org.apache.metamodel.schema.Table;
+
+public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
+
+    private final HBaseColumn[] _outputColumns;
+
+    public HBaseRowInsertionBuilder(HBaseUpdateCallback updateCallback, Table table, HBaseColumn[] outputColumns) {
+        super(updateCallback, table, outputColumns.length);
+        _outputColumns = outputColumns;
+    }
+
+    @Override
+    public void execute() throws MetaModelException {
+        checkForMatchingColumnFamilies(getTable(), _outputColumns);
+        getUpdateCallback().writeRow((HBaseTable) getTable(), _outputColumns, getValues());
+    }
+
+    private void checkForMatchingColumnFamilies(Table table, HBaseColumn[] outputColumns) {
+        for (int i = 0; i < outputColumns.length; i++) {
+            if (!outputColumns[i].getColumnFamily().equals(HBaseDataContext.FIELD_ID)) {
+                boolean matchingColumnFound = false;
+                int indexOfTablesColumn = 0;
+
+                while (!matchingColumnFound && indexOfTablesColumn < table.getColumnCount()) {
+                    if (outputColumns[i].getColumnFamily().equals(table.getColumn(indexOfTablesColumn).getName())) {
+                        matchingColumnFound = true;
+                    } else {
+                        indexOfTablesColumn++;
+                    }
+                }
+
+                if (!matchingColumnFound) {
+                    throw new IllegalArgumentException(String.format(
+                            "OutputColumnFamily: %s doesn't exist in the schema of the table", outputColumns[i]
+                                    .getColumnFamily()));
+                }
+            }
+        }
+    }
+
+    public HBaseColumn[] getOutputColumns() {
+        return _outputColumns;
+    }
+
+    public void setOutputColumns(HBaseColumn[] outputColumns) {
+        if (outputColumns.length != _outputColumns.length) {
+            throw new IllegalArgumentException("The amount of outputColumns don't match");
+        }
+        for (int i = 0; i < outputColumns.length; i++) {
+            _outputColumns[i] = outputColumns[i];
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
new file mode 100644
index 0000000..5ab38d9
--- /dev/null
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
@@ -0,0 +1,37 @@
+/**
+ * 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.metamodel.hbase;
+
+import org.apache.metamodel.MetaModelException;
+import org.apache.metamodel.drop.AbstractTableDropBuilder;
+import org.apache.metamodel.schema.Table;
+
+public class HBaseTableDropBuilder extends AbstractTableDropBuilder {
+    private final HBaseUpdateCallback _updateCallback;
+
+    public HBaseTableDropBuilder(Table table, HBaseUpdateCallback updateCallback) {
+        super(table);
+        _updateCallback = updateCallback;
+    }
+
+    @Override
+    public void execute() throws MetaModelException {
+        _updateCallback.dropTableExecute(getTable());
+    }
+}

http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
new file mode 100644
index 0000000..271aa87
--- /dev/null
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
@@ -0,0 +1,134 @@
+/**
+ * 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.metamodel.hbase;
+
+import java.io.IOException;
+
+import org.apache.metamodel.AbstractUpdateCallback;
+import org.apache.metamodel.UpdateCallback;
+import org.apache.metamodel.create.TableCreationBuilder;
+import org.apache.metamodel.delete.RowDeletionBuilder;
+import org.apache.metamodel.drop.TableDropBuilder;
+import org.apache.metamodel.insert.RowInsertionBuilder;
+import org.apache.metamodel.schema.MutableSchema;
+import org.apache.metamodel.schema.Schema;
+import org.apache.metamodel.schema.Table;
+
+public class HBaseUpdateCallback extends AbstractUpdateCallback implements UpdateCallback {
+
+    private final HBaseConfiguration _configuration;
+
+    private final HBaseDataContext _dataContext;
+
+    public HBaseUpdateCallback(HBaseDataContext dataContext) {
+        super(dataContext);
+        _configuration = dataContext.getConfiguration();
+        _dataContext = dataContext;
+    }
+
+    @Override
+    public TableCreationBuilder createTable(Schema schema, String name) throws IllegalArgumentException,
+            IllegalStateException {
+        throw new UnsupportedOperationException(
+                "Use createTable(Schema schema, String name, HBaseColumn[] outputColumns)");
+    }
+
+    public HBaseCreateTableBuilder createTable(Schema schema, String name, HBaseColumn[] outputColumns)
+            throws IllegalArgumentException, IllegalStateException {
+        return new HBaseCreateTableBuilder(this, schema, name, outputColumns);
+    }
+
+    @Override
+    public boolean isDropTableSupported() {
+        return true;
+    }
+
+    @Override
+    public TableDropBuilder dropTable(Table table) throws IllegalArgumentException, IllegalStateException,
+            UnsupportedOperationException {
+        return new HBaseTableDropBuilder(table, this);
+    }
+
+    public void dropTableExecute(Table table) {
+        try {
+            final HBaseWriter HbaseWriter = new HBaseWriter(HBaseDataContext.createConfig(_configuration));
+            HbaseWriter.dropTable(table.getName());
+            MutableSchema schema = (MutableSchema) table.getSchema();
+            schema.removeTable(table);
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+
+    @Override
+    public RowInsertionBuilder insertInto(Table table) throws IllegalArgumentException, IllegalStateException,
+            UnsupportedOperationException {
+        throw new UnsupportedOperationException("Use insertInto(String tableName, HBaseColumn[] outputColumns)");
+    }
+
+    public HBaseRowInsertionBuilder insertInto(String tableName, HBaseColumn[] outputColumns)
+            throws IllegalArgumentException, IllegalStateException, UnsupportedOperationException {
+        Table table = getTable(tableName);
+        return insertInto(table, outputColumns);
+    }
+
+    public HBaseRowInsertionBuilder insertInto(Table table, HBaseColumn[] outputColumns)
+            throws IllegalArgumentException, IllegalStateException, UnsupportedOperationException {
+        validateTable(table);
+        return new HBaseRowInsertionBuilder(this, table, outputColumns);
+    }
+
+    private void validateTable(Table table) {
+        if (!(table instanceof HBaseTable)) {
+            throw new IllegalArgumentException("Not a valid HBase table: " + table);
+        }
+    }
+
+    protected synchronized void writeRow(HBaseTable hBaseTable, HBaseColumn[] outputColumns, Object[] values) {
+        try {
+            final HBaseWriter HbaseWriter = new HBaseWriter(HBaseDataContext.createConfig(_configuration));
+            HbaseWriter.writeRow(hBaseTable, outputColumns, values);
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+
+    @Override
+    public boolean isDeleteSupported() {
+        return false;
+    }
+
+    @Override
+    public RowDeletionBuilder deleteFrom(Table table) throws IllegalArgumentException, IllegalStateException,
+            UnsupportedOperationException {
+        throw new UnsupportedOperationException();
+    }
+
+    public boolean tableAlreadyExists(String tableName) {
+        return _dataContext.getMainSchema().getTableByName(tableName) == null ? false : true;
+    }
+
+    public HBaseConfiguration getConfiguration() {
+        return _configuration;
+    }
+
+    public HBaseDataContext getDataContext() {
+        return _dataContext;
+    }
+}

http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/hbase/src/main/java/org/apache/metamodel/hbase/HBaseWriter.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseWriter.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseWriter.java
new file mode 100644
index 0000000..f8897da
--- /dev/null
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseWriter.java
@@ -0,0 +1,138 @@
+/**
+ * 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.metamodel.hbase;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metamodel.MetaModelException;
+
+public final class HBaseWriter extends Configured {
+
+    static final byte[] INFO_COLUMNFAMILY = Bytes.toBytes("info");
+    static final byte[] NAME_QUALIFIER = Bytes.toBytes("name");
+    static final byte[] LOCATION_QUALIFIER = Bytes.toBytes("location");
+    static final byte[] DESCRIPTION_QUALIFIER = Bytes.toBytes("description");
+
+    private final Connection _connection;
+
+    public HBaseWriter(Configuration configuration) throws IOException {
+        _connection = ConnectionFactory.createConnection(configuration);
+    }
+
+    public void writeRow(HBaseTable hBaseTable, HBaseColumn[] outputColumns, Object[] values) throws IOException {
+        try {
+            Table table = _connection.getTable(TableName.valueOf(hBaseTable.getName()));
+            try {
+                int indexOfIdColumn = 0;
+                boolean idColumnFound = false;
+                while (!idColumnFound && indexOfIdColumn < outputColumns.length) {
+                    if (outputColumns[indexOfIdColumn].getColumnFamily().equals(HBaseDataContext.FIELD_ID)) {
+                        idColumnFound = true;
+                    } else {
+                        indexOfIdColumn++;
+                    }
+                }
+                if (!idColumnFound) {
+                    throw new MetaModelException("The ID Column family was not found");
+                }
+
+                Put put = new Put(Bytes.toBytes(values[indexOfIdColumn].toString()));
+
+                for (int i = 0; i < outputColumns.length; i++) {
+                    if (!outputColumns[i].getColumnFamily().equals(HBaseDataContext.FIELD_ID)) {
+                        put.addColumn(Bytes.toBytes(outputColumns[i].getColumnFamily()), Bytes.toBytes(outputColumns[i]
+                                .getQualifier()), Bytes.toBytes(values[i].toString()));
+                    }
+                }
+                table.put(put);
+            } finally {
+                table.close();
+            }
+        } finally {
+            _connection.close();
+        }
+    }
+
+    public void deleteRow(HBaseTable hBaseTable, Object key) throws IOException {
+        try {
+            Table table = _connection.getTable(TableName.valueOf(hBaseTable.getName()));
+            try {
+                table.delete(new Delete(Bytes.toBytes(key.toString())));
+            } finally {
+                table.close();
+            }
+        } finally {
+            _connection.close();
+        }
+    }
+
+    public void createTable(String tableName, Set<String> columnFamilies) throws IOException {
+        try {
+            // Create table
+            Admin admin = _connection.getAdmin();
+            try {
+                TableName hBasetableName = TableName.valueOf(tableName);
+                HTableDescriptor tableDescriptor = new HTableDescriptor(hBasetableName);
+                for (String columnFamilie : columnFamilies) {
+                    if (!columnFamilie.equals(HBaseDataContext.FIELD_ID)) {
+                        tableDescriptor.addFamily(new HColumnDescriptor(columnFamilie));
+                    }
+                }
+                admin.createTable(tableDescriptor);
+                HTableDescriptor[] tables = admin.listTables();
+                if (tables.length != 1 && Bytes.equals(hBasetableName.getName(), tables[0].getTableName().getName())) {
+                    throw new IOException("Failed create of table");
+                }
+            } finally {
+                admin.close();
+            }
+        } finally {
+            _connection.close();
+        }
+
+    }
+
+    public void dropTable(String tableName) throws IOException {
+        try {
+            Admin admin = _connection.getAdmin();
+            try {
+                TableName hBasetableName = TableName.valueOf(tableName);
+                admin.disableTable(hBasetableName);
+                admin.deleteTable(hBasetableName);
+            } finally {
+                admin.close();
+            }
+        } finally {
+            _connection.close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/metamodel/blob/6ad602bb/hbase/src/main/java/org/apache/metamodel/hbase/qualifiers/FindQualifiersDriver.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/qualifiers/FindQualifiersDriver.java b/hbase/src/main/java/org/apache/metamodel/hbase/qualifiers/FindQualifiersDriver.java
new file mode 100644
index 0000000..3efa311
--- /dev/null
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/qualifiers/FindQualifiersDriver.java
@@ -0,0 +1,115 @@
+/**
+ * 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.metamodel.hbase.qualifiers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
+import org.apache.hadoop.hbase.mapreduce.TableMapper;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+public class FindQualifiersDriver extends Configured implements Tool {
+
+    static class OnlyColumnNameMapper extends TableMapper<Text, Text> {
+        @Override
+        protected void map(ImmutableBytesWritable key, Result value, final Context context) throws IOException,
+                InterruptedException {
+            CellScanner cellScanner = value.cellScanner();
+            while (cellScanner.advance()) {
+
+                Cell cell = cellScanner.current();
+                byte[] q = Bytes.copy(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
+
+                context.write(new Text(q), new Text());
+            }
+        }
+    }
+
+    static class OnlyColumnNameReducer extends Reducer<Text, Text, Text, Text> {
+
+        @Override
+        protected void reduce(Text key, Iterable<Text> values, Context context) throws IOException,
+                InterruptedException {
+            context.write(new Text(key), new Text());
+        }
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Path outputPath = new Path("output/");
+        byte[] tableName = new String("ietsanders2").getBytes();
+        byte[] columnFamilyName = new String("data").getBytes();
+
+        Configuration configuration = createConfig();
+        FileSystem fileSystem = FileSystem.get(configuration);
+        fileSystem.delete(outputPath, true);
+
+        Job job = Job.getInstance(configuration, "Distinct_columns");
+        job.setJarByClass(this.getClass());
+
+        Scan scan = new Scan();
+        scan.setBatch(500);
+        scan.addFamily(columnFamilyName);
+        scan.setFilter(new KeyOnlyFilter()); // scan only key part of KeyValue (raw, column family, column)
+        scan.setCacheBlocks(false); // don't set to true for MR jobs
+
+        TextOutputFormat.setOutputPath(job, outputPath);
+
+        TableMapReduceUtil.initTableMapperJob(tableName, scan, OnlyColumnNameMapper.class, // mapper
+                Text.class, // mapper output key
+                Text.class, // mapper output value
+                job);
+
+        job.setNumReduceTasks(1);
+        job.setReducerClass(OnlyColumnNameReducer.class);
+
+        return job.waitForCompletion(true) ? 0 : 1;
+    }
+
+    protected Configuration createConfig() {
+        Configuration config = org.apache.hadoop.hbase.HBaseConfiguration.create();
+        config.set("hbase.zookeeper.quorum", "bigdatavm");
+        config.set("hbase.zookeeper.property.clientPort", Integer.toString(2181));
+        config.set("hbase.client.retries.number", Integer.toString(1));
+        config.set("zookeeper.session.timeout", Integer.toString(5000));
+        config.set("zookeeper.recovery.retry", Integer.toString(1));
+        return config;
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new FindQualifiersDriver(), args);
+        System.exit(exitCode);
+    }
+}


[21/31] metamodel git commit: - Reverted newly introduced constructor for AbstractRowInsertionBuilder. - Refactored HBaseUpdateCallback so it more strictly adheres to the MetaModel UpdateCallback interface. It is no longer public. - Removed some of the o

Posted by ka...@apache.org.
- Reverted newly introduced constructor for AbstractRowInsertionBuilder.
- Refactored HBaseUpdateCallback so it more strictly adheres to the MetaModel UpdateCallback interface. It is no longer public.
- Removed some of the overabundant constructors for HBaseColumn and changed the ID column's column type to ColumnType.BINARY.
- Refactored the HBaseRowInsertionBuilder, so it can be instantiated without the columns for the update. Move all possible "value" methods there, because it can't determine based on the Table which columns are available for updating (because Big Table doesn't work in that manner), you can use the "value" method to dynamically add HBase columns for inserting.
- Updated unit tests accordingly.


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/909cb48f
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/909cb48f
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/909cb48f

Branch: refs/heads/master
Commit: 909cb48ff31be6476b71e9565826f6579446acce
Parents: 5313a49
Author: Arjan Seijkens <a....@quadient.com>
Authored: Thu Jun 7 17:10:41 2018 +0200
Committer: Arjan Seijkens <a....@quadient.com>
Committed: Thu Jun 7 17:10:41 2018 +0200

----------------------------------------------------------------------
 .../insert/AbstractRowInsertionBuilder.java     |   6 -
 .../org/apache/metamodel/hbase/HBaseColumn.java |  22 +---
 .../hbase/HBaseRowInsertionBuilder.java         | 112 ++++++++++++-----
 .../metamodel/hbase/HBaseUpdateCallback.java    |  20 +---
 .../apache/metamodel/hbase/DeleteRowTest.java   |   7 +-
 .../hbase/HBaseUpdateCallbackTest.java          |  11 +-
 .../apache/metamodel/hbase/InsertRowTest.java   | 120 ++-----------------
 7 files changed, 104 insertions(+), 194 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/909cb48f/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java b/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
index bf6dcd9..58c0da8 100644
--- a/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
+++ b/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
@@ -44,12 +44,6 @@ public abstract class AbstractRowInsertionBuilder<U extends UpdateCallback> exte
         _table = table;
     }
 
-    public AbstractRowInsertionBuilder(final U updateCallback, final Table table, final List<Column> columns) {
-        super(columns);
-        _updateCallback = updateCallback;
-        _table = table;
-    }
-
     @Override
     public Table getTable() {
         return _table;

http://git-wip-us.apache.org/repos/asf/metamodel/blob/909cb48f/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
index 45e7f4f..d07c477 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
@@ -19,38 +19,27 @@
 package org.apache.metamodel.hbase;
 
 import org.apache.metamodel.schema.ColumnType;
-import org.apache.metamodel.schema.ColumnTypeImpl;
 import org.apache.metamodel.schema.MutableColumn;
-import org.apache.metamodel.schema.SuperColumnType;
 import org.apache.metamodel.schema.Table;
 
 public final class HBaseColumn extends MutableColumn {
-    public static final ColumnType DEFAULT_COLUMN_TYPE_FOR_ID_COLUMN = new ColumnTypeImpl("BYTE[]",
-            SuperColumnType.LITERAL_TYPE);
+    public static final ColumnType DEFAULT_COLUMN_TYPE_FOR_ID_COLUMN = ColumnType.BINARY;
     public static final ColumnType DEFAULT_COLUMN_TYPE_FOR_COLUMN_FAMILIES = ColumnType.LIST;
 
     private final String columnFamily;
     private final String qualifier;
 
     public HBaseColumn(final String columnFamily, final Table table) {
-        this(columnFamily, null, table, -1);
+        this(columnFamily, null, table, -1, null);
     }
 
     public HBaseColumn(final String columnFamily, final String qualifier, final Table table) {
-        this(columnFamily, qualifier, table, -1);
-    }
-
-    public HBaseColumn(final String columnFamily, final Table table, final int columnNumber) {
-        this(columnFamily, null, table, columnNumber);
-    }
-
-    public HBaseColumn(final String columnFamily, final String qualifier, final Table table, final int columnNumber) {
-        this(columnFamily, qualifier, table, columnNumber, null);
+        this(columnFamily, qualifier, table, -1, null);
     }
 
     public HBaseColumn(final String columnFamily, final String qualifier, final Table table, final int columnNumber,
             final ColumnType columnType) {
-        super(columnFamily, table);
+        super(getName(columnFamily, qualifier), table);
         if (columnFamily == null) {
             throw new IllegalArgumentException("Column family isn't allowed to be null.");
         } else if (table == null || !(table instanceof HBaseTable)) {
@@ -83,8 +72,7 @@ public final class HBaseColumn extends MutableColumn {
         return qualifier;
     }
 
-    @Override
-    public String getName() {
+    private static String getName(final String columnFamily, final String qualifier) {
         if (qualifier == null) {
             return columnFamily;
         }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/909cb48f/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
index b6f797a..0a3a4a3 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
@@ -18,6 +18,7 @@
  */
 package org.apache.metamodel.hbase;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
@@ -26,16 +27,19 @@ import java.util.Set;
 import java.util.stream.Collectors;
 
 import org.apache.metamodel.MetaModelException;
+import org.apache.metamodel.data.Style;
 import org.apache.metamodel.insert.AbstractRowInsertionBuilder;
+import org.apache.metamodel.insert.RowInsertionBuilder;
 import org.apache.metamodel.schema.Column;
 
 /**
  * A builder-class to insert rows in a HBase datastore.
  */
-// TODO: Possible future improvement: Make it possible to change the columns for each execute.
-// Now each row will get exactly the same columns.
 public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
-    private final int _indexOfIdColumn;
+    private List<HBaseColumn> columns = new ArrayList<>();
+    private List<Object> values = new ArrayList<>();
+
+    private int _indexOfIdColumn = -1;
 
     /**
      * Creates a {@link HBaseRowInsertionBuilder}. The table and the column's columnFamilies are checked to exist in the schema.
@@ -45,33 +49,10 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
      * @throws IllegalArgumentException the columns list can't be null or empty
      * @throws MetaModelException when no ID-column is found.
      */
-    public HBaseRowInsertionBuilder(final HBaseUpdateCallback updateCallback, final HBaseTable table,
-            final List<HBaseColumn> columns) {
-        super(updateCallback, table, columns.stream().map(column -> (Column) column).collect(Collectors.toList()));
-
-        this._indexOfIdColumn = getIndexOfIdColumn(columns);
-        if (_indexOfIdColumn == -1) {
-            throw new MetaModelException("The ID-Column was not found");
-        }
+    public HBaseRowInsertionBuilder(final HBaseUpdateCallback updateCallback, final HBaseTable table) {
+        super(updateCallback, table);
 
         checkTable(updateCallback, table);
-        // The columns parameter should match the table's columns, just to be sure, this is checked again
-        checkColumnFamilies(table, getColumnFamilies(columns));
-    }
-
-    /**
-     * Returns the index of the ID-column (see {@link HBaseDataContext#FIELD_ID}) in an array of HBaseColumns.
-     *
-     * @param columns
-     * @return index of the ID-column
-     */
-    private static int getIndexOfIdColumn(final List<HBaseColumn> columns) {
-        for (int i = 0; i < columns.size(); i++) {
-            if (HBaseDataContext.FIELD_ID.equals(columns.get(i).getColumnFamily())) {
-                return i;
-            }
-        }
-        return -1;
     }
 
     /**
@@ -121,19 +102,84 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
      * @param columns
      * @return {@link LinkedHashSet}
      */
-    private static Set<String> getColumnFamilies(final List<HBaseColumn> columns) {
-        return columns.stream().map(HBaseColumn::getColumnFamily).distinct().collect(Collectors.toSet());
+    private static Set<String> getColumnFamilies(final HBaseColumn[] columns) {
+        return Arrays.stream(columns).map(HBaseColumn::getColumnFamily).distinct().collect(Collectors.toSet());
     }
 
     @Override
     public synchronized void execute() {
+        if (_indexOfIdColumn == -1) {
+            throw new MetaModelException("The ID-Column was not found");
+        }
+
+        // The columns parameter should match the table's columns, just to be sure, this is checked again
+        checkColumnFamilies((HBaseTable) getTable(), getColumnFamilies(getColumns()));
+
         ((HBaseDataContext) getUpdateCallback().getDataContext()).getHBaseClient().insertRow(getTable().getName(),
                 getColumns(), getValues(), _indexOfIdColumn);
     }
 
     @Override
-    public HBaseColumn[] getColumns() {
-        return Arrays.stream(super.getColumns()).map(column -> (HBaseColumn) column).toArray(
-                size -> new HBaseColumn[size]);
+    protected HBaseColumn[] getColumns() {
+        return columns.toArray(new HBaseColumn[columns.size()]);
+    }
+
+    @Override
+    protected Object[] getValues() {
+        return values.toArray(new Object[values.size()]);
+    }
+
+    @Override
+    public RowInsertionBuilder value(final Column column, final Object value, final Style style) {
+        if (column == null) {
+            throw new IllegalArgumentException("Column cannot be null.");
+        }
+        if (!(column instanceof HBaseColumn)) {
+            throw new IllegalArgumentException("Column is not an HBaseColumn.");
+        }
+
+        for (int i = 0; i < columns.size(); i++) {
+            if (columns.get(i).equals(column)) {
+                values.set(i, value);
+                return this;
+            }
+        }
+
+        if (column.isPrimaryKey()) {
+            _indexOfIdColumn = columns.size();
+        }
+
+        columns.add((HBaseColumn) column);
+        values.add(value);
+
+        return this;
+    }
+
+    @Override
+    public RowInsertionBuilder value(final int columnIndex, final Object value) {
+        values.set(columnIndex, value);
+        return this;
+    }
+
+    @Override
+    public RowInsertionBuilder value(final String columnName, final Object value) {
+        for (Column column : columns) {
+            if (column.getName().equals(columnName)) {
+                return value(column, value, null);
+            }
+        }
+
+        throw new IllegalArgumentException("No such column in table: " + columnName + ", available columns are: "
+                + columns);
+    }
+
+    @Override
+    public boolean isSet(final Column column) {
+        for (int i = 0; i < columns.size(); i++) {
+            if (columns.get(i).equals(column)) {
+                return values.get(i) != null;
+            }
+        }
+        return false;
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/909cb48f/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
index 9c17ddd..5f44200 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
@@ -18,7 +18,6 @@
  */
 package org.apache.metamodel.hbase;
 
-import java.util.List;
 import java.util.Set;
 
 import org.apache.metamodel.AbstractUpdateCallback;
@@ -33,7 +32,7 @@ import org.apache.metamodel.schema.Table;
 /**
  * This class is used to build objects to do client-operations on a HBase datastore
  */
-public class HBaseUpdateCallback extends AbstractUpdateCallback implements UpdateCallback {
+final class HBaseUpdateCallback extends AbstractUpdateCallback implements UpdateCallback {
 
     public HBaseUpdateCallback(final HBaseDataContext dataContext) {
         super(dataContext);
@@ -71,23 +70,8 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
      */
     @Override
     public RowInsertionBuilder insertInto(final Table table) {
-        throw new UnsupportedOperationException(
-                "We need an explicit list of columns when inserting into an HBase table.");
-    }
-
-    /**
-     * Initiates the building of a row insertion operation.
-     * @param table Table to get inserts.
-     * @param columns List of {@link HBaseColumn} to insert on.
-     * @return {@link HBaseRowInsertionBuilder}
-     * @throws IllegalArgumentException The table must be an {@link HBaseTable} and the columns list can't be null or empty
-     */
-    public HBaseRowInsertionBuilder insertInto(final Table table, final List<HBaseColumn> columns) {
-        if (columns == null || columns.isEmpty()) {
-            throw new IllegalArgumentException("The hbaseColumns list is null or empty");
-        }
         if (table instanceof HBaseTable) {
-            return new HBaseRowInsertionBuilder(this, (HBaseTable) table, columns);
+            return new HBaseRowInsertionBuilder(this, (HBaseTable) table);
         } else {
             throw new IllegalArgumentException("Not an HBase table: " + table);
         }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/909cb48f/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
index 7aeb938..5a95032 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
@@ -18,13 +18,13 @@
  */
 package org.apache.metamodel.hbase;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.List;
 import java.util.Map;
 
 import org.apache.metamodel.MetaModelException;
+import org.apache.metamodel.insert.RowInsertionBuilder;
 import org.apache.metamodel.schema.MutableTable;
 import org.junit.Rule;
 import org.junit.Test;
@@ -152,10 +152,9 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
         final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                 CF_BAR);
         final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
-        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
 
         checkRows(false, false);
-        final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
+        final RowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable);
         setValuesInInsertionBuilder(row, rowInsertionBuilder);
         rowInsertionBuilder.execute();
         checkRows(true, false);

http://git-wip-us.apache.org/repos/asf/metamodel/blob/909cb48f/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
index 0d279b8..14205fb 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metamodel.insert.RowInsertionBuilder;
 import org.apache.metamodel.schema.ColumnType;
 import org.apache.metamodel.schema.MutableSchema;
 import org.apache.metamodel.schema.Table;
@@ -84,7 +85,7 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
 
     /**
      * Check if the table has been inserted successfully. Checks are performed in the schema and the datastore.
-     * 
+     *
      * @throws IOException because the admin object needs to be created
      */
     protected void checkSuccesfullyInsertedTable() throws IOException {
@@ -212,12 +213,8 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
      * @param enoughMatchingValues if true, the amount of columns match the amount of values
      */
     protected void setValuesInInsertionBuilder(final Map<HBaseColumn, Object> row,
-            final HBaseRowInsertionBuilder rowInsertionBuilder) {
-        int i = 0;
-        for (Object value : row.values()) {
-            rowInsertionBuilder.value(i, value);
-            i++;
-        }
+            final RowInsertionBuilder rowInsertionBuilder) {
+        row.entrySet().forEach(entry -> rowInsertionBuilder.value(entry.getKey(), entry.getValue()));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/metamodel/blob/909cb48f/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
index 61e8f9c..31a33d0 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
@@ -21,12 +21,12 @@ package org.apache.metamodel.hbase;
 import static org.junit.Assert.*;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.metamodel.MetaModelException;
+import org.apache.metamodel.insert.RowInsertionBuilder;
 import org.apache.metamodel.schema.MutableTable;
 import org.junit.Rule;
 import org.junit.Test;
@@ -47,21 +47,6 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     }
 
     /**
-     * Using only the table parameter, should throw an exception
-     *
-     * @throws IOException
-     */
-    @Test
-    public void testOnlyUsingTableParameter() throws IOException {
-        exception.expect(UnsupportedOperationException.class);
-        exception.expectMessage("We need an explicit list of columns when inserting into an HBase table.");
-
-        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-        getUpdateCallback().insertInto(existingTable);
-    }
-
-    /**
      * Having the table type wrong, should throw an exception
      *
      * @throws IOException
@@ -72,40 +57,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
         exception.expect(IllegalArgumentException.class);
         exception.expectMessage("Not an HBase table: " + mutableTable);
 
-        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                CF_BAR);
-        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
-        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-        getUpdateCallback().insertInto(mutableTable, columns);
-    }
-
-    /**
-     * Having the columns parameter null at the updateCallBack, should throw an exception
-     *
-     * @throws IOException
-     */
-    @Test
-    public void testColumnsNullAtUpdateCallBack() throws IOException {
-        exception.expect(IllegalArgumentException.class);
-        exception.expectMessage("The hbaseColumns list is null or empty");
-
-        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                CF_BAR);
-        getUpdateCallback().insertInto(existingTable, null);
-    }
-
-    /**
-     * Having the columns parameter empty at the updateCallBack, should throw an exception
-     *
-     * @throws IOException
-     */
-    @Test
-    public void testColumnsEmptyAtUpdateCallBack() throws IOException {
-        exception.expect(IllegalArgumentException.class);
-        exception.expectMessage("The hbaseColumns list is null or empty");
-        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                CF_BAR);
-        getUpdateCallback().insertInto(existingTable, new ArrayList<HBaseColumn>());
+        getUpdateCallback().insertInto(mutableTable);
     }
 
     /**
@@ -121,11 +73,9 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
         exception.expectMessage("Trying to insert data into table: " + wrongTable.getName()
                 + ", which doesn't exist yet");
 
-        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                CF_BAR);
-        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
-        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-        getUpdateCallback().insertInto(wrongTable, columns);
+        createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+
+        getUpdateCallback().insertInto(wrongTable);
     }
 
     /**
@@ -139,54 +89,10 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
         exception.expectMessage("The ID-Column was not found");
 
         final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-        final Map<HBaseColumn, Object> row = createRow(existingTable, null, CF_FOO, CF_BAR, false);
-        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-        getUpdateCallback().insertInto(existingTable, columns);
-    }
-
-    /**
-     * If the column family doesn't exist in the table (wrong columnFamily), then a exception should be thrown
-     *
-     * @throws IOException
-     */
-    @Test
-    public void testColumnFamilyDoesntExistsBecauseItsNull() throws IOException {
-        final String wrongColumnFamily = "wrongColumnFamily";
-
-        exception.expect(MetaModelException.class);
-        exception.expectMessage(String.format("ColumnFamily: %s doesn't exist in the schema of the table",
-                wrongColumnFamily));
-
-        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                 CF_BAR);
-        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
-        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-        final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                wrongColumnFamily);
-        getUpdateCallback().insertInto(wrongTable, columns);
-    }
 
-    /**
-     * If the column family doesn't exist in the table (new columnFamily), then a exception should be thrown
-     *
-     * @throws IOException
-     */
-    @Test
-    public void testColumnFamilyDoesntExistsBecauseItsNew() throws IOException {
-        final String wrongColumnFamily = "newColumnFamily";
-
-        exception.expect(MetaModelException.class);
-        exception.expectMessage(String.format("ColumnFamily: %s doesn't exist in the schema of the table",
-                wrongColumnFamily));
-
-        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                CF_BAR);
-        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
-        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-        final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
-                wrongColumnFamily);
-        getUpdateCallback().insertInto(wrongTable, columns);
+        final RowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable);
+        rowInsertionBuilder.execute();
     }
 
     /**
@@ -300,9 +206,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     public void testInsertIntoWithoutExecute() throws IOException {
         final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                 CF_BAR);
-        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
-        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-        getUpdateCallback().insertInto(existingTable, columns);
+        getUpdateCallback().insertInto(existingTable);
     }
 
     /**
@@ -311,14 +215,13 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
      * @throws IOException
      */
     @Test
-    public void testQaulifierNull() throws IOException {
+    public void testQualifierNull() throws IOException {
         final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                 CF_BAR);
         final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, true);
-        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
 
         checkRows(false, true);
-        final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
+        final RowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable);
         setValuesInInsertionBuilder(row, rowInsertionBuilder);
         rowInsertionBuilder.execute();
         checkRows(true, true);
@@ -334,10 +237,9 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
         final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                 CF_BAR);
         final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
-        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
 
         checkRows(false, false);
-        final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
+        final RowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable);
         setValuesInInsertionBuilder(row, rowInsertionBuilder);
         rowInsertionBuilder.execute();
         checkRows(true, false);


[12/31] metamodel git commit: HBase unittests improvements (fixed issues when the testfile wasn't setup)

Posted by ka...@apache.org.
HBase unittests improvements (fixed issues when the testfile wasn't setup)


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/54a900b4
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/54a900b4
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/54a900b4

Branch: refs/heads/master
Commit: 54a900b4c9a608687239982d602c8eabc67f3afa
Parents: 9ab3004
Author: Gerard Dellemann <g....@quadient.com>
Authored: Wed May 30 10:54:22 2018 +0200
Committer: Gerard Dellemann <g....@quadient.com>
Committed: Wed May 30 10:54:22 2018 +0200

----------------------------------------------------------------------
 .../apache/metamodel/hbase/CreateTableTest.java | 122 ++++++++++++-------
 .../apache/metamodel/hbase/DeleteRowTest.java   |  54 +++++---
 .../apache/metamodel/hbase/DropTableTest.java   |  22 +++-
 .../apache/metamodel/hbase/HBaseTestCase.java   |  40 +++---
 .../hbase/HBaseUpdateCallbackTest.java          |  22 ++--
 .../apache/metamodel/hbase/InsertRowTest.java   |   9 +-
 6 files changed, 169 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/54a900b4/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
index e804a67..5655e4a 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
@@ -31,19 +31,29 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      * Check if creating table is supported
      */
     public void testDropTableSupported() {
-        assertTrue(getUpdateCallback().isCreateTableSupported());
+        if (isConfigured()) {
+            assertTrue(getUpdateCallback().isCreateTableSupported());
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
     }
 
     /**
      * Create a table with an immutableSchema, should throw a IllegalArgumentException
      */
     public void testWrongSchema() {
-        final ImmutableSchema immutableSchema = new ImmutableSchema(getSchema());
-        try {
-            getUpdateCallback().createTable(immutableSchema, TABLE_NAME).execute();
-            fail("Should get an exception that the schema isn't mutable");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Not a mutable schema: " + immutableSchema, e.getMessage());
+        if (isConfigured()) {
+            final ImmutableSchema immutableSchema = new ImmutableSchema(getSchema());
+            try {
+                getUpdateCallback().createTable(immutableSchema, TABLE_NAME).execute();
+                fail("Should get an exception that the schema isn't mutable");
+            } catch (IllegalArgumentException e) {
+                assertEquals("Not a mutable schema: " + immutableSchema, e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
         }
     }
 
@@ -51,11 +61,16 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      * Create a table without columnFamilies, should throw a MetaModelException
      */
     public void testCreateTableWithoutColumnFamilies() {
-        try {
-            getUpdateCallback().createTable(getSchema(), TABLE_NAME).execute();
-            fail("Should get an exception that the columnFamilies haven't been set");
-        } catch (MetaModelException e) {
-            assertEquals("Creating a table without columnFamilies", e.getMessage());
+        if (isConfigured()) {
+            try {
+                getUpdateCallback().createTable(getSchema(), TABLE_NAME).execute();
+                fail("Should get an exception that the columnFamilies haven't been set");
+            } catch (MetaModelException e) {
+                assertEquals("Creating a table without columnFamilies", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
         }
     }
 
@@ -63,11 +78,16 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      * Create a table with columnFamilies null, should throw a MetaModelException
      */
     public void testColumnFamiliesNull() {
-        try {
-            getUpdateCallback().createTable(getSchema(), TABLE_NAME, null).execute();
-            fail("Should get an exception that the columnFamilies haven't been set");
-        } catch (MetaModelException e) {
-            assertEquals("Creating a table without columnFamilies", e.getMessage());
+        if (isConfigured()) {
+            try {
+                getUpdateCallback().createTable(getSchema(), TABLE_NAME, null).execute();
+                fail("Should get an exception that the columnFamilies haven't been set");
+            } catch (MetaModelException e) {
+                assertEquals("Creating a table without columnFamilies", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
         }
     }
 
@@ -75,12 +95,17 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      * Create a table with columnFamilies empty, should throw a MetaModelException
      */
     public void testColumnFamiliesEmpty() {
-        try {
-            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<String>();
-            getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
-            fail("Should get an exception that the columnFamilies haven't been set");
-        } catch (MetaModelException e) {
-            assertEquals("Creating a table without columnFamilies", e.getMessage());
+        if (isConfigured()) {
+            try {
+                final LinkedHashSet<String> columnFamilies = new LinkedHashSet<String>();
+                getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
+                fail("Should get an exception that the columnFamilies haven't been set");
+            } catch (MetaModelException e) {
+                assertEquals("Creating a table without columnFamilies", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
         }
     }
 
@@ -112,13 +137,18 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      * Creating a HBaseClient with the tableName null, should throw a exception
      */
     public void testCreatingTheHBaseClientWithTableNameNull() {
-        try {
-            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
-            columnFamilies.add("1");
-            new HBaseClient(getDataContext().getConnection()).createTable(null, columnFamilies);
-            fail("Should get an exception that tableName is null");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
+        if (isConfigured()) {
+            try {
+                final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
+                columnFamilies.add("1");
+                new HBaseClient(getDataContext().getConnection()).createTable(null, columnFamilies);
+                fail("Should get an exception that tableName is null");
+            } catch (IllegalArgumentException e) {
+                assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
         }
     }
 
@@ -126,11 +156,16 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      * Creating a HBaseClient with the tableName null, should throw a exception
      */
     public void testCreatingTheHBaseClientWithColumnFamiliesNull() {
-        try {
-            new HBaseClient(getDataContext().getConnection()).createTable("1", null);
-            fail("Should get an exception that columnFamilies is null");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
+        if (isConfigured()) {
+            try {
+                new HBaseClient(getDataContext().getConnection()).createTable("1", null);
+                fail("Should get an exception that columnFamilies is null");
+            } catch (IllegalArgumentException e) {
+                assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
         }
     }
 
@@ -138,12 +173,17 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      * Creating a HBaseClient with the tableName null, should throw a exception
      */
     public void testCreatingTheHBaseClientWithColumnFamiliesEmpty() {
-        try {
-            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
-            new HBaseClient(getDataContext().getConnection()).createTable("1", columnFamilies);
-            fail("Should get an exception that columnFamilies is empty");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
+        if (isConfigured()) {
+            try {
+                final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
+                new HBaseClient(getDataContext().getConnection()).createTable("1", columnFamilies);
+                fail("Should get an exception that columnFamilies is empty");
+            } catch (IllegalArgumentException e) {
+                assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/54a900b4/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
index 8d21194..893b956 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
@@ -31,19 +31,29 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
      * Delete is supported
      */
     public void testDeleteSupported() {
-        assertTrue(getUpdateCallback().isDeleteSupported());
+        if (isConfigured()) {
+            assertTrue(getUpdateCallback().isDeleteSupported());
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
     }
 
     /**
      * Having the table type wrong, should throw an exception
      */
     public void testTableWrongType() {
-        final MutableTable mutableTable = new MutableTable();
-        try {
-            getUpdateCallback().deleteFrom(mutableTable);
-            fail("Should get an exception that the type of the table is wrong.");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Not an HBase table: " + mutableTable, e.getMessage());
+        if (isConfigured()) {
+            final MutableTable mutableTable = new MutableTable();
+            try {
+                getUpdateCallback().deleteFrom(mutableTable);
+                fail("Should get an exception that the type of the table is wrong.");
+            } catch (IllegalArgumentException e) {
+                assertEquals("Not an HBase table: " + mutableTable, e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
         }
     }
 
@@ -91,11 +101,16 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
      * Creating a HBaseClient with the tableName null, should throw a exception
      */
     public void testCreatingTheHBaseClientWithTableNameNull() {
-        try {
-            new HBaseClient(getDataContext().getConnection()).deleteRow(null, new String("1"));
-            fail("Should get an exception that tableName is null");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't delete a row without having tableName or rowKey", e.getMessage());
+        if (isConfigured()) {
+            try {
+                new HBaseClient(getDataContext().getConnection()).deleteRow(null, new String("1"));
+                fail("Should get an exception that tableName is null");
+            } catch (IllegalArgumentException e) {
+                assertEquals("Can't delete a row without having tableName or rowKey", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
         }
     }
 
@@ -103,11 +118,16 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
      * Creating a HBaseClient with the rowKey null, should throw a exception
      */
     public void testCreatingTheHBaseClientWithRowKeyNull() {
-        try {
-            new HBaseClient(getDataContext().getConnection()).deleteRow("tableName", null);
-            fail("Should get an exception that rowKey is null");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't delete a row without having tableName or rowKey", e.getMessage());
+        if (isConfigured()) {
+            try {
+                new HBaseClient(getDataContext().getConnection()).deleteRow("tableName", null);
+                fail("Should get an exception that rowKey is null");
+            } catch (IllegalArgumentException e) {
+                assertEquals("Can't delete a row without having tableName or rowKey", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/54a900b4/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
index d5b19ef..39bb06d 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
@@ -30,7 +30,12 @@ public class DropTableTest extends HBaseUpdateCallbackTest {
      * Check if drop table is supported
      */
     public void testDropTableSupported() {
-        assertTrue(getUpdateCallback().isDropTableSupported());
+        if (isConfigured()) {
+            assertTrue(getUpdateCallback().isDropTableSupported());
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
     }
 
     /**
@@ -55,11 +60,16 @@ public class DropTableTest extends HBaseUpdateCallbackTest {
      * Creating a HBaseClient with the tableName null, should throw a exception
      */
     public void testCreatingTheHBaseClientWithTableNameNull() {
-        try {
-            new HBaseClient(getDataContext().getConnection()).dropTable(null);
-            fail("Should get an exception that tableName is null");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't drop a table without having the tableName", e.getMessage());
+        if (isConfigured()) {
+            try {
+                new HBaseClient(getDataContext().getConnection()).dropTable(null);
+                fail("Should get an exception that tableName is null");
+            } catch (IllegalArgumentException e) {
+                assertEquals("Can't drop a table without having the tableName", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/54a900b4/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
index 91180cb..70fd2e3 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
@@ -63,40 +63,34 @@ public abstract class HBaseTestCase extends TestCase {
     private boolean _configured;
     private static HBaseDataContext _dataContext;
 
-    private boolean setUpIsDone = false;
-
     @Override
     protected void setUp() throws Exception {
         super.setUp();
 
-        if (!setUpIsDone) {
-            Properties properties = new Properties();
-            File file = new File(getPropertyFilePath());
-            if (file.exists()) {
-                properties.load(new FileReader(file));
-                zookeeperHostname = properties.getProperty("hbase.zookeeper.hostname");
-                String zookeeperPortPropertyValue = properties.getProperty("hbase.zookeeper.port");
-                if (zookeeperPortPropertyValue != null && !zookeeperPortPropertyValue.isEmpty()) {
-                    zookeeperPort = Integer.parseInt(zookeeperPortPropertyValue);
-                }
-
-                _configured = (zookeeperHostname != null && !zookeeperHostname.isEmpty());
-            } else {
-                _configured = false;
-            }
-            if (isConfigured()) {
-                final HBaseConfiguration configuration = new HBaseConfiguration(zookeeperHostname, zookeeperPort,
-                        ColumnType.VARCHAR);
-                setDataContext(new HBaseDataContext(configuration));
+        Properties properties = new Properties();
+        File file = new File(getPropertyFilePath());
+        if (file.exists()) {
+            properties.load(new FileReader(file));
+            zookeeperHostname = properties.getProperty("hbase.zookeeper.hostname");
+            String zookeeperPortPropertyValue = properties.getProperty("hbase.zookeeper.port");
+            if (zookeeperPortPropertyValue != null && !zookeeperPortPropertyValue.isEmpty()) {
+                zookeeperPort = Integer.parseInt(zookeeperPortPropertyValue);
             }
-            setUpIsDone = true;
+
+            _configured = (zookeeperHostname != null && !zookeeperHostname.isEmpty());
+        } else {
+            _configured = false;
+        }
+        if (isConfigured()) {
+            final HBaseConfiguration configuration = new HBaseConfiguration(zookeeperHostname, zookeeperPort,
+                    ColumnType.VARCHAR);
+            setDataContext(new HBaseDataContext(configuration));
         }
     }
 
     @AfterClass
     public static void oneTimeTeardown() throws IOException {
         _dataContext.getConnection().close();
-        ;
     }
 
     private String getPropertyFilePath() {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/54a900b4/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
index b010c71..60a5ba6 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
@@ -43,19 +43,19 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
     private HBaseUpdateCallback updateCallback;
     private MutableSchema schema;
 
-    private boolean setUpIsDone = false;
+    private static boolean warningGiven = false;
 
     @Override
     protected void setUp() throws Exception {
         super.setUp();
         if (isConfigured()) {
-            if (setUpIsDone) {
-                dropTableIfItExists();
-            } else {
-                updateCallback = new HBaseUpdateCallback(getDataContext());
-                schema = (MutableSchema) getDataContext().getDefaultSchema();
-                dropTableIfItExists();
-                setUpIsDone = true;
+            updateCallback = new HBaseUpdateCallback(getDataContext());
+            schema = (MutableSchema) getDataContext().getDefaultSchema();
+            dropTableIfItExists();
+        } else {
+            if (!warningGiven) {
+                System.err.println(getInvalidConfigurationMessage());
+                warningGiven = true;
             }
         }
     }
@@ -187,9 +187,9 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
     }
 
     protected void warnAboutANotExecutedTest(String className, String methodName) {
-        String logWarning = "Test(method) \"" + className + "#" + methodName
-                + "\" is not executed, because the HBasetest is not configured.";
-        // System.out.println(logWarning);
+        String logWarning = "Test \"" + className + "#" + methodName
+                + "()\" is not executed, because the HBasetest is not configured.";
+        System.err.println(logWarning);
         logger.warn(logWarning);
     }
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/54a900b4/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
index 622b70b..7a1705f 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
@@ -34,7 +34,12 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
      * @throws IOException
      */
     public void testInsertSupported() throws IOException {
-        assertTrue(getUpdateCallback().isInsertSupported());
+        if (isConfigured()) {
+            assertTrue(getUpdateCallback().isInsertSupported());
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
     }
 
     /**
@@ -63,7 +68,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
      */
     public void testWrongTableType() throws IOException {
         if (isConfigured()) {
-            MutableTable mutableTable = new MutableTable();
+            final MutableTable mutableTable = new MutableTable();
             try {
                 final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);


[05/31] metamodel git commit: HBase improvements - Added row-deletion functionality, also made HBaseDataContext implement UpdateableDataContext

Posted by ka...@apache.org.
HBase improvements - Added row-deletion functionality, also made HBaseDataContext implement UpdateableDataContext


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/0ffeeb20
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/0ffeeb20
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/0ffeeb20

Branch: refs/heads/master
Commit: 0ffeeb20af4d58061d042bb6ebccafb2ab7d6502
Parents: f2d2ff2
Author: Gerard Dellemann <g....@quadient.com>
Authored: Tue May 1 14:34:36 2018 +0200
Committer: Gerard Dellemann <g....@quadient.com>
Committed: Tue May 1 14:34:36 2018 +0200

----------------------------------------------------------------------
 .../metamodel/hbase/HBaseDataContext.java       | 13 ++++-
 .../org/apache/metamodel/hbase/HBaseRow.java    |  5 --
 .../hbase/HBaseRowDeletionBuilder.java          | 50 ++++++++++++++++++++
 .../metamodel/hbase/HBaseUpdateCallback.java    | 25 ++++++++--
 .../org/apache/metamodel/hbase/HBaseWriter.java | 19 ++++++--
 5 files changed, 97 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/0ffeeb20/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
index 841b225..b913ca0 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
@@ -35,6 +35,9 @@ import org.apache.hadoop.hbase.filter.PageFilter;
 import org.apache.metamodel.DataContext;
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.QueryPostprocessDataContext;
+import org.apache.metamodel.UpdateScript;
+import org.apache.metamodel.UpdateSummary;
+import org.apache.metamodel.UpdateableDataContext;
 import org.apache.metamodel.data.DataSet;
 import org.apache.metamodel.data.DataSetHeader;
 import org.apache.metamodel.data.Row;
@@ -53,7 +56,7 @@ import org.slf4j.LoggerFactory;
 /**
  * MetaModel adaptor for Apache HBase.
  */
-public class HBaseDataContext extends QueryPostprocessDataContext {
+public class HBaseDataContext extends QueryPostprocessDataContext implements UpdateableDataContext {
 
     private static final Logger logger = LoggerFactory.getLogger(HBaseDataContext.class);
 
@@ -243,4 +246,12 @@ public class HBaseDataContext extends QueryPostprocessDataContext {
     private void setMaxRows(Scan scan, int maxRows) {
         scan.setFilter(new PageFilter(maxRows));
     }
+
+    @Override
+    public UpdateSummary executeUpdate(UpdateScript update) {
+        final HBaseUpdateCallback callback = new HBaseUpdateCallback(this);
+        update.run(callback);
+
+        return callback.getUpdateSummary();
+    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/0ffeeb20/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java
index 2820602..79267da 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java
@@ -61,11 +61,6 @@ final class HBaseRow extends AbstractRow implements Row {
         }
         final int colonIndex = name.indexOf(':');
         if (colonIndex != -1) {
-            /*
-             * I think this is DEATH code! The first line of this method: implementation
-             * _header.getSelectItem(index).getColumn() will always return only a columnfamily with our current
-             * implementations.
-             */
             byte[] family = name.substring(0, colonIndex).getBytes();
             byte[] qualifier = name.substring(colonIndex + 1).getBytes();
             byte[] value = _result.getValue(family, qualifier);

http://git-wip-us.apache.org/repos/asf/metamodel/blob/0ffeeb20/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
new file mode 100644
index 0000000..925537a
--- /dev/null
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
@@ -0,0 +1,50 @@
+/**
+ * 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.metamodel.hbase;
+
+import org.apache.metamodel.MetaModelException;
+import org.apache.metamodel.delete.AbstractRowDeletionBuilder;
+import org.apache.metamodel.schema.Table;
+
+public class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
+
+    private HBaseUpdateCallback _updateCallback;
+    private Object _key;
+
+    public HBaseRowDeletionBuilder(HBaseUpdateCallback updateCallback, Table table) {
+        super(table);
+        if (updateCallback == null) {
+            throw new IllegalArgumentException("UpdateCallback cannot be null");
+        }
+        _updateCallback = updateCallback;
+
+    }
+
+    @Override
+    public void execute() throws MetaModelException {
+        if (_key == null) {
+            throw new IllegalArgumentException("Key cannot be null");
+        }
+        _updateCallback.deleteRow((HBaseTable) getTable(), _key);
+    }
+
+    public void setKey(Object _key) {
+        this._key = _key;
+    }
+}

http://git-wip-us.apache.org/repos/asf/metamodel/blob/0ffeeb20/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
index 6cc1522..39670db 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
@@ -79,11 +79,10 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
     @Override
     public RowInsertionBuilder insertInto(Table table) throws IllegalArgumentException, IllegalStateException,
             UnsupportedOperationException {
-        throw new UnsupportedOperationException("Use insertInto(String tableName, HBaseColumn[] outputColumns)");
+        throw new UnsupportedOperationException("Use insertInto(Table table, HBaseColumn[] outputColumns)");
     }
 
-    public HBaseRowInsertionBuilder insertInto(Table table, HBaseColumn[] columns)
-            throws IllegalArgumentException {
+    public HBaseRowInsertionBuilder insertInto(Table table, HBaseColumn[] columns) throws IllegalArgumentException {
         if (table instanceof HBaseTable) {
             return new HBaseRowInsertionBuilder(this, (HBaseTable) table, columns);
         }
@@ -107,7 +106,25 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
     @Override
     public RowDeletionBuilder deleteFrom(Table table) throws IllegalArgumentException, IllegalStateException,
             UnsupportedOperationException {
-        throw new UnsupportedOperationException();
+        throw new UnsupportedOperationException(
+                "Use deleteFrom(HBaseUpdateCallback updateCallback, Table table, Object key)");
+    }
+
+    public HBaseRowDeletionBuilder deleteFrom(HBaseUpdateCallback updateCallback, Table table)
+            throws IllegalArgumentException {
+        if (table instanceof HBaseTable) {
+            return new HBaseRowDeletionBuilder(this, (HBaseTable) table);
+        }
+        throw new IllegalArgumentException("Not an HBase table: " + table);
+    }
+
+    protected synchronized void deleteRow(HBaseTable hBaseTable, Object key) {
+        try {
+            final HBaseWriter HbaseWriter = new HBaseWriter(HBaseDataContext.createConfig(_configuration));
+            HbaseWriter.deleteRow(hBaseTable, key);
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
     }
 
     public HBaseConfiguration getConfiguration() {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/0ffeeb20/hbase/src/main/java/org/apache/metamodel/hbase/HBaseWriter.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseWriter.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseWriter.java
index f8897da..ad63597 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseWriter.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseWriter.java
@@ -30,17 +30,17 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.metamodel.MetaModelException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public final class HBaseWriter extends Configured {
 
-    static final byte[] INFO_COLUMNFAMILY = Bytes.toBytes("info");
-    static final byte[] NAME_QUALIFIER = Bytes.toBytes("name");
-    static final byte[] LOCATION_QUALIFIER = Bytes.toBytes("location");
-    static final byte[] DESCRIPTION_QUALIFIER = Bytes.toBytes("description");
+    private static final Logger logger = LoggerFactory.getLogger(HBaseWriter.class);
 
     private final Connection _connection;
 
@@ -86,7 +86,11 @@ public final class HBaseWriter extends Configured {
         try {
             Table table = _connection.getTable(TableName.valueOf(hBaseTable.getName()));
             try {
-                table.delete(new Delete(Bytes.toBytes(key.toString())));
+                if (rowExists(table, key) == true) {
+                    table.delete(new Delete(Bytes.toBytes(key.toString())));
+                } else {
+                    logger.warn("Rowkey with value " + key.toString() + " doesn't exist in the table");
+                }
             } finally {
                 table.close();
             }
@@ -95,6 +99,11 @@ public final class HBaseWriter extends Configured {
         }
     }
 
+    private boolean rowExists(Table table, Object key) throws IOException {
+        Get get = new Get(Bytes.toBytes(key.toString()));
+        return (table.get(get).isEmpty()) == true ? false : true;
+    }
+
     public void createTable(String tableName, Set<String> columnFamilies) throws IOException {
         try {
             // Create table


[02/31] metamodel git commit: Small improvements and fixed the existing Unit Test

Posted by ka...@apache.org.
Small improvements and fixed the existing Unit Test


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/f15d0272
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/f15d0272
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/f15d0272

Branch: refs/heads/master
Commit: f15d02721910678e29c1452c1bf6f4b495609efc
Parents: 6ad602b
Author: HUMANINFERENCE\g.dellemann <g....@quadient.com>
Authored: Mon Apr 23 16:53:03 2018 +0200
Committer: HUMANINFERENCE\g.dellemann <g....@quadient.com>
Committed: Mon Apr 23 16:53:03 2018 +0200

----------------------------------------------------------------------
 hbase/pom.xml                                   |  6 ++
 .../metamodel/hbase/HBaseDataContext.java       |  6 --
 .../hbase/qualifiers/FindQualifiersDriver.java  |  9 +-
 .../metamodel/hbase/HBaseDataContextTest.java   | 98 ++++++++++++++------
 4 files changed, 80 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/f15d0272/hbase/pom.xml
----------------------------------------------------------------------
diff --git a/hbase/pom.xml b/hbase/pom.xml
index aed4f63..47101ba 100644
--- a/hbase/pom.xml
+++ b/hbase/pom.xml
@@ -264,6 +264,12 @@
 			<artifactId>slf4j-log4j12</artifactId>
 			<scope>provided</scope>
 		</dependency>
+		<dependency>
+			<groupId>io.netty</groupId>
+			<artifactId>netty-transport</artifactId>
+			<version>4.1.13.Final</version>
+			<scope>test</scope>
+		</dependency>
 
 		<!-- Test dependencies -->
 		<dependency>

http://git-wip-us.apache.org/repos/asf/metamodel/blob/f15d0272/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
index 6b48723..841b225 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
@@ -72,9 +72,6 @@ public class HBaseDataContext extends QueryPostprocessDataContext {
         Configuration config = createConfig(configuration);
         _configuration = configuration;
         _connection = createConnection(config);
-
-        // HBaseUpdateCallback updateCallback = new HBaseUpdateCallback(this);
-        // updateCallback.writeRow();
     }
 
     /**
@@ -87,9 +84,6 @@ public class HBaseDataContext extends QueryPostprocessDataContext {
         super(false);
         _configuration = configuration;
         _connection = connection;
-
-        // HBaseUpdateCallback updateCallback = new HBaseUpdateCallback(this);
-        // updateCallback.writeRow();
     }
 
     private Connection createConnection(Configuration config) {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/f15d0272/hbase/src/main/java/org/apache/metamodel/hbase/qualifiers/FindQualifiersDriver.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/qualifiers/FindQualifiersDriver.java b/hbase/src/main/java/org/apache/metamodel/hbase/qualifiers/FindQualifiersDriver.java
index 3efa311..9292d06 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/qualifiers/FindQualifiersDriver.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/qualifiers/FindQualifiersDriver.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.metamodel.hbase.HBaseConfiguration;
 
 public class FindQualifiersDriver extends Configured implements Tool {
 
@@ -101,10 +102,10 @@ public class FindQualifiersDriver extends Configured implements Tool {
     protected Configuration createConfig() {
         Configuration config = org.apache.hadoop.hbase.HBaseConfiguration.create();
         config.set("hbase.zookeeper.quorum", "bigdatavm");
-        config.set("hbase.zookeeper.property.clientPort", Integer.toString(2181));
-        config.set("hbase.client.retries.number", Integer.toString(1));
-        config.set("zookeeper.session.timeout", Integer.toString(5000));
-        config.set("zookeeper.recovery.retry", Integer.toString(1));
+        config.set("hbase.zookeeper.property.clientPort", Integer.toString(HBaseConfiguration.DEFAULT_ZOOKEEPER_PORT));
+        config.set("hbase.client.retries.number", Integer.toString(HBaseConfiguration.DEFAULT_HBASE_CLIENT_RETRIES));
+        config.set("zookeeper.session.timeout", Integer.toString(HBaseConfiguration.DEFAULT_ZOOKEEPER_SESSION_TIMEOUT));
+        config.set("zookeeper.recovery.retry", Integer.toString(HBaseConfiguration.DEFAULT_ZOOKEEPER_RECOVERY_RETRIES));
         return config;
     }
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/f15d0272/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
index 6187f2c..fb21b46 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
@@ -32,8 +32,32 @@ import org.apache.metamodel.util.SimpleTableDef;
 
 public class HBaseDataContextTest extends HBaseTestCase {
 
+    // Table
     private static final String EXAMPLE_TABLE_NAME = "table_for_junit";
 
+    // ColumnFamilies
+    private static final String CF_FOO = "foo";
+    private static final String CF_BAR = "bar";
+
+    // Qualifiers
+    private static final String Q_HELLO = "hello";
+    private static final String Q_HI = "hi";
+    private static final String Q_HEY = "hey";
+    private static final String Q_BAH = "bah";
+
+    // RowKeys
+    private static final String RK_1 = "junit1";
+    private static final String RK_2 = "junit2";
+
+    private static final int NUMBER_OF_ROWS = 2;
+
+    // Values
+    private static final String V_WORLD = "world";
+    private static final String V_THERE = "there";
+    private static final String V_YO = "yo";
+    private static final byte[] V_123_BYTE_ARRAY = new byte[] { 1, 2, 3 };
+    private static final String V_YOU = "you";
+
     private HBaseDataContext _dataContext;
 
     @Override
@@ -59,7 +83,9 @@ public class HBaseDataContextTest extends HBaseTestCase {
         final Table table = _dataContext.getDefaultSchema().getTableByName(EXAMPLE_TABLE_NAME);
         assertNotNull(table);
 
-        assertEquals("[_id, bar, foo]", Arrays.toString(table.getColumnNames().toArray()));
+        assertEquals("[" + HBaseDataContext.FIELD_ID + ", " + CF_BAR + ", " + CF_FOO + "]", Arrays.toString(table
+                .getColumnNames()
+                .toArray()));
         assertEquals(ColumnType.MAP, table.getColumn(1).getType());
 
         // insert two records
@@ -69,33 +95,39 @@ public class HBaseDataContextTest extends HBaseTestCase {
         final DataSet dataSet1 = _dataContext.query().from(EXAMPLE_TABLE_NAME).selectAll().execute();
         try {
             assertTrue(dataSet1.next());
-            assertEquals(
-                    "Row[values=[junit1, {[104, 101, 121]=[121, 111],[104, 105]=[116, 104, 101, 114, 101]}, {[104, 101, 108, 108, 111]=[119, 111, 114, 108, 100]}]]",
-                    dataSet1.getRow().toString());
+            assertEquals("Row[values=[" + RK_1 + ", {" + Q_HEY + "=" + V_YO + "," + Q_HI + "=" + V_THERE + "}, {"
+                    + Q_HELLO + "=" + V_WORLD + "}]]", dataSet1.getRow().toString());
             assertTrue(dataSet1.next());
-            assertEquals("Row[values=[junit2, {[98, 97, 104]=[1, 2, 3],[104, 105]=[121, 111, 117]}, {}]]", dataSet1
-                    .getRow().toString());
+            assertEquals("Row[values=[" + RK_2 + ", {" + Q_BAH + "=" + new String(V_123_BYTE_ARRAY) + "," + Q_HI + "="
+                    + V_YOU + "}, {}]]", dataSet1.getRow().toString());
             assertFalse(dataSet1.next());
         } finally {
             dataSet1.close();
         }
 
         // query using custom table definitions
-        final String[] columnNames = new String[] { "foo", "bar:hi", "bar:hey" };
+        final String columnName1 = CF_FOO;
+        final String columnName2 = CF_BAR + ":" + Q_HI;
+        final String columnName3 = CF_BAR + ":" + Q_HEY;
+        final String[] columnNames = new String[] { columnName1, columnName2, columnName3 };
         final ColumnType[] columnTypes = new ColumnType[] { ColumnType.MAP, ColumnType.VARCHAR, ColumnType.VARCHAR };
         final SimpleTableDef[] tableDefinitions = new SimpleTableDef[] { new SimpleTableDef(EXAMPLE_TABLE_NAME,
                 columnNames, columnTypes) };
         _dataContext = new HBaseDataContext(new HBaseConfiguration("SCH", getZookeeperHostname(), getZookeeperPort(),
                 tableDefinitions, ColumnType.VARCHAR));
 
-        final DataSet dataSet2 = _dataContext.query().from(EXAMPLE_TABLE_NAME).select("foo", "bar:hi", "bar:hey")
+        final DataSet dataSet2 = _dataContext
+                .query()
+                .from(EXAMPLE_TABLE_NAME)
+                .select(columnName1, columnName2, columnName3)
                 .execute();
         try {
             assertTrue(dataSet2.next());
-            assertEquals("Row[values=[{[104, 101, 108, 108, 111]=[119, 111, 114, 108, 100]}, there, yo]]", dataSet2
-                    .getRow().toString());
+            assertEquals("Row[values=[{" + Q_HELLO + "=" + V_WORLD + "}, " + V_THERE + ", " + V_YO + "]]", dataSet2
+                    .getRow()
+                    .toString());
             assertTrue(dataSet2.next());
-            assertEquals("Row[values=[{}, you, null]]", dataSet2.getRow().toString());
+            assertEquals("Row[values=[{}, " + V_YOU + ", null]]", dataSet2.getRow().toString());
             assertFalse(dataSet2.next());
         } finally {
             dataSet2.close();
@@ -105,33 +137,41 @@ public class HBaseDataContextTest extends HBaseTestCase {
         final DataSet dataSet3 = _dataContext.query().from(EXAMPLE_TABLE_NAME).selectCount().execute();
         try {
             assertTrue(dataSet3.next());
-            assertEquals("Row[values=[2]]", dataSet3.getRow().toString());
+            assertEquals("Row[values=[" + NUMBER_OF_ROWS + "]]", dataSet3.getRow().toString());
             assertFalse(dataSet3.next());
         } finally {
             dataSet3.close();
         }
 
         // query only id
-        final DataSet dataSet4 = _dataContext.query().from(EXAMPLE_TABLE_NAME).select(HBaseDataContext.FIELD_ID)
+        final DataSet dataSet4 = _dataContext
+                .query()
+                .from(EXAMPLE_TABLE_NAME)
+                .select(HBaseDataContext.FIELD_ID)
                 .execute();
 
         try {
             assertTrue(dataSet4.next());
-            assertEquals("Row[values=[junit1]]", dataSet4.getRow().toString());
+            assertEquals("Row[values=[" + RK_1 + "]]", dataSet4.getRow().toString());
             assertTrue(dataSet4.next());
-            assertEquals("Row[values=[junit2]]", dataSet4.getRow().toString());
+            assertEquals("Row[values=[" + RK_2 + "]]", dataSet4.getRow().toString());
             assertFalse(dataSet4.next());
         } finally {
             dataSet4.close();
         }
 
         // primary key lookup query - using GET
-        final DataSet dataSet5 = _dataContext.query().from(EXAMPLE_TABLE_NAME).select(HBaseDataContext.FIELD_ID)
-                .where(HBaseDataContext.FIELD_ID).eq("junit1").execute();
+        final DataSet dataSet5 = _dataContext
+                .query()
+                .from(EXAMPLE_TABLE_NAME)
+                .select(HBaseDataContext.FIELD_ID)
+                .where(HBaseDataContext.FIELD_ID)
+                .eq(RK_1)
+                .execute();
 
         try {
             assertTrue(dataSet5.next());
-            assertEquals("Row[values=[junit1]]", dataSet5.getRow().toString());
+            assertEquals("Row[values=[" + RK_1 + "]]", dataSet5.getRow().toString());
             assertFalse(dataSet5.next());
         } finally {
             dataSet5.close();
@@ -141,16 +181,16 @@ public class HBaseDataContextTest extends HBaseTestCase {
     private void insertRecordsNatively() throws Exception {
         final org.apache.hadoop.hbase.client.Table hTable = _dataContext.getHTable(EXAMPLE_TABLE_NAME);
         try {
-            final Put put1 = new Put("junit1".getBytes());
-            put1.addColumn("foo".getBytes(), "hello".getBytes(), "world".getBytes());
-            put1.addColumn("bar".getBytes(), "hi".getBytes(), "there".getBytes());
-            put1.addColumn("bar".getBytes(), "hey".getBytes(), "yo".getBytes());
+            final Put put1 = new Put(RK_1.getBytes());
+            put1.addColumn(CF_FOO.getBytes(), Q_HELLO.getBytes(), V_WORLD.getBytes());
+            put1.addColumn(CF_BAR.getBytes(), Q_HI.getBytes(), V_THERE.getBytes());
+            put1.addColumn(CF_BAR.getBytes(), Q_HEY.getBytes(), V_YO.getBytes());
 
-            final Put put2 = new Put("junit2".getBytes());
-            put2.addColumn("bar".getBytes(), "bah".getBytes(), new byte[] { 1, 2, 3 });
-            put2.addColumn("bar".getBytes(), "hi".getBytes(), "you".getBytes());
+            final Put put2 = new Put(RK_2.getBytes());
+            put2.addColumn(CF_BAR.getBytes(), Q_BAH.getBytes(), V_123_BYTE_ARRAY);
+            put2.addColumn(CF_BAR.getBytes(), Q_HI.getBytes(), V_YOU.getBytes());
 
-            final Object[] result = new Object[2];
+            final Object[] result = new Object[NUMBER_OF_ROWS];
             hTable.batch(Arrays.asList(put1, put2), result);
         } finally {
             hTable.close();
@@ -159,7 +199,7 @@ public class HBaseDataContextTest extends HBaseTestCase {
 
     private void createTableNatively() throws Exception {
         final TableName tableName = TableName.valueOf(EXAMPLE_TABLE_NAME);
-        
+
         // check if the table exists
         if (_dataContext.getAdmin().isTableAvailable(tableName)) {
             System.out.println("Unittest table already exists: " + EXAMPLE_TABLE_NAME);
@@ -170,8 +210,8 @@ public class HBaseDataContextTest extends HBaseTestCase {
         Admin admin = _dataContext.getAdmin();
         System.out.println("Creating table");
         final HTableDescriptor tableDescriptor = new HTableDescriptor(tableName);
-        tableDescriptor.addFamily(new HColumnDescriptor("foo".getBytes()));
-        tableDescriptor.addFamily(new HColumnDescriptor("bar".getBytes()));
+        tableDescriptor.addFamily(new HColumnDescriptor(CF_FOO.getBytes()));
+        tableDescriptor.addFamily(new HColumnDescriptor(CF_BAR.getBytes()));
         admin.createTable(tableDescriptor);
         System.out.println("Created table");
     }


[04/31] metamodel git commit: Merge pull request #1 from arjansh/feature/hbase-improvements-refactoring

Posted by ka...@apache.org.
Merge pull request #1 from arjansh/feature/hbase-improvements-refactoring

Reverting some core changes and refactoring HBaseColumn class

Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/f2d2ff2a
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/f2d2ff2a
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/f2d2ff2a

Branch: refs/heads/master
Commit: f2d2ff2a503ceebbd0bd3493d4df60cae08126ae
Parents: f15d027 2008c7e
Author: Gerard Dellemann <ge...@gmail.com>
Authored: Mon Apr 30 16:08:06 2018 +0200
Committer: GitHub <no...@github.com>
Committed: Mon Apr 30 16:08:06 2018 +0200

----------------------------------------------------------------------
 .../metamodel/AbstractUpdateCallback.java       |   4 +-
 .../metamodel/data/AbstractRowBuilder.java      |  33 +-----
 .../insert/AbstractRowInsertionBuilder.java     |  10 +-
 hbase/pom.xml                                   | 111 ------------------
 .../org/apache/metamodel/hbase/HBaseColumn.java | 110 +++++++++++++++---
 .../hbase/HBaseRowInsertionBuilder.java         |   6 +-
 .../metamodel/hbase/HBaseUpdateCallback.java    |  24 +---
 .../hbase/qualifiers/FindQualifiersDriver.java  | 116 -------------------
 8 files changed, 110 insertions(+), 304 deletions(-)
----------------------------------------------------------------------



[16/31] metamodel git commit: HBase improvements (when creating a HBase table, the ID-column is not a required parameter, because that's always created).

Posted by ka...@apache.org.
HBase improvements (when creating a HBase table, the ID-column is not a required parameter, because that's always created).


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/f4d2c97a
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/f4d2c97a
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/f4d2c97a

Branch: refs/heads/master
Commit: f4d2c97ad3a7fa41934251e6fd727639ed1bd300
Parents: cd755a1
Author: Gerard Dellemann <g....@quadient.com>
Authored: Thu May 31 16:30:20 2018 +0200
Committer: Gerard Dellemann <g....@quadient.com>
Committed: Thu May 31 16:30:20 2018 +0200

----------------------------------------------------------------------
 .../hbase/HBaseCreateTableBuilder.java          | 26 ++--------
 .../apache/metamodel/hbase/CreateTableTest.java | 52 ++++++++++----------
 .../hbase/HBaseUpdateCallbackTest.java          | 13 +++--
 3 files changed, 40 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/f4d2c97a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
index 138e32b..db4463b 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
@@ -18,7 +18,6 @@
  */
 package org.apache.metamodel.hbase;
 
-import java.util.Iterator;
 import java.util.Set;
 
 import org.apache.metamodel.MetaModelException;
@@ -58,7 +57,9 @@ public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseU
 
     @Override
     public Table execute() {
-        checkColumnFamilies(_columnFamilies);
+        if (_columnFamilies == null || _columnFamilies.size() == 0) {
+            throw new MetaModelException("Creating a table without columnFamilies");
+        }
 
         final Table table = getTable();
 
@@ -71,27 +72,6 @@ public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseU
     }
 
     /**
-     * Check if the new table has columnFamilies and if the ID-column is included.
-     * Throws a {@link MetaModelException} if a check fails.
-     * @param columnFamilies
-     */
-    private void checkColumnFamilies(Set<String> columnFamilies) {
-        if (columnFamilies == null || columnFamilies.size() == 0) {
-            throw new MetaModelException("Creating a table without columnFamilies");
-        }
-        boolean idColumnFound = false;
-        final Iterator<String> iterator = columnFamilies.iterator();
-        while (!idColumnFound && iterator.hasNext()) {
-            if (iterator.next().equals(HBaseDataContext.FIELD_ID)) {
-                idColumnFound = true;
-            }
-        }
-        if (!idColumnFound) {
-            throw new MetaModelException("ColumnFamily: " + HBaseDataContext.FIELD_ID + " not found");
-        }
-    }
-
-    /**
      * Set the columnFamilies
      * @param columnFamilies
      */

http://git-wip-us.apache.org/repos/asf/metamodel/blob/f4d2c97a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
index 789b4bd..6c39acd 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.metamodel.hbase;
 
+import java.io.IOException;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
@@ -32,7 +33,7 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
     /**
      * Check if creating table is supported
      */
-    public void testDropTableSupported() {
+    public void testCreateTableSupported() {
         if (isConfigured()) {
             assertTrue(getUpdateCallback().isCreateTableSupported());
         } else {
@@ -112,30 +113,6 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
     }
 
     /**
-     * Create a table without the ID-Column, should throw a MetaModelException
-     */
-    public void testCreateTableWithoutIDColumn() {
-        if (isConfigured()) {
-            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, null, CF_FOO, CF_BAR);
-            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
-            try {
-                final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
-                        .createTable(getSchema(), TABLE_NAME);
-
-                hBaseCreateTableBuilder.setColumnFamilies(columnFamilies);
-                hBaseCreateTableBuilder.execute();
-                fail("Should get an exception that the ID-colum is missing");
-            } catch (MetaModelException e) {
-                assertEquals("ColumnFamily: " + HBaseDataContext.FIELD_ID + " not found", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
-        }
-    }
-
-    /**
      * Creating a HBaseClient with the tableName null, should throw a exception
      */
     public void testCreatingTheHBaseClientWithTableNameNull() {
@@ -190,6 +167,31 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
     }
 
     /**
+     * Goodflow. Create a table without the ID-Column, should work
+     * @throws IOException 
+     */
+    public void testCreateTableWithoutIDColumn() throws IOException {
+        if (isConfigured()) {
+            final HBaseTable table = createHBaseTable(TABLE_NAME, null, CF_FOO, CF_BAR, null);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, null, CF_FOO, CF_BAR);
+            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
+            try {
+                final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
+                        .createTable(getSchema(), TABLE_NAME);
+
+                hBaseCreateTableBuilder.setColumnFamilies(columnFamilies);
+                hBaseCreateTableBuilder.execute();
+                checkSuccesfullyInsertedTable();
+            } catch (Exception e) {
+                fail("Should not get an exception (that the ID-column is missing)");
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
      * Goodflow. Create a table including the ID-Column (columnFamilies not in constructor), should work
      */
     public void testSettingColumnFamiliesAfterConstrutor() {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/f4d2c97a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
index 5085689..a30b7f2 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
@@ -126,7 +126,7 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
     /**
      * Create a test HBase table
      * @param tableName name of the table
-     * @param idColumn required ID-column
+     * @param idColumn ID-column, can be set to null to create a table without this column
      * @param columnFamily1 required columnFamily 1
      * @param columnFamily2 required columnFamily 2
      * @param columnFamily3 columnFamily 3 is not required and can be used to test errors
@@ -136,9 +136,16 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
             final String columnFamily2, final String columnFamily3) {
         String[] columnNames;
         ColumnType[] columnTypes;
-        if (columnFamily3 == null) {
+
+        if (idColumn == null && columnFamily3 == null) {
+            columnNames = new String[] { columnFamily1, columnFamily2 };
+            columnTypes = new ColumnType[] { ColumnType.STRING, ColumnType.STRING };
+        } else if (idColumn != null && columnFamily3 == null) {
             columnNames = new String[] { idColumn, columnFamily1, columnFamily2 };
             columnTypes = new ColumnType[] { ColumnType.STRING, ColumnType.STRING, ColumnType.STRING };
+        } else if (idColumn == null && columnFamily3 != null) {
+            columnNames = new String[] { columnFamily1, columnFamily2, columnFamily3 };
+            columnTypes = new ColumnType[] { ColumnType.STRING, ColumnType.STRING, ColumnType.STRING };
         } else {
             columnNames = new String[] { idColumn, columnFamily1, columnFamily2, columnFamily3 };
             columnTypes = new ColumnType[] { ColumnType.STRING, ColumnType.STRING, ColumnType.STRING,
@@ -151,7 +158,7 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
     /**
      * Creates a map that represents a row
      * @param table HBaseTable
-     * @param idColumn required ID-column
+     * @param idColumn ID-column, can be set to null to create a row without this column
      * @param columnFamily1 required columnFamily 1
      * @param columnFamily2 required columnFamily 1
      * @return {@link LinkedHashMap}<{@link HBaseColumn}, {@link Object}>


[25/31] metamodel git commit: Throw IllegalArgumentException when trying to delete a row with an empty row key.

Posted by ka...@apache.org.
Throw IllegalArgumentException when trying to delete a row with an empty row key.


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/0ca1fd2f
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/0ca1fd2f
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/0ca1fd2f

Branch: refs/heads/master
Commit: 0ca1fd2fa8d8cc7b2e9cdad20485ece626f44a0c
Parents: 146277c
Author: Arjan Seijkens <a....@quadient.com>
Authored: Wed Jun 13 14:49:15 2018 +0200
Committer: Arjan Seijkens <a....@quadient.com>
Committed: Wed Jun 13 14:49:15 2018 +0200

----------------------------------------------------------------------
 .../src/main/java/org/apache/metamodel/hbase/HBaseClient.java | 2 +-
 .../test/java/org/apache/metamodel/hbase/DeleteRowTest.java   | 7 +++++--
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/0ca1fd2f/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
index a868a93..0594a5b 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
@@ -118,7 +118,7 @@ final class HBaseClient {
                 throw new MetaModelException(e);
             }
         } else {
-            logger.info("Have not deleted a row, which has an empty (\"\") rowKey.");
+            throw new IllegalArgumentException("Can't delete a row without an empty rowKey.");
         }
     }
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/0ca1fd2f/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
index 2eefb06..4e588ca 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
@@ -103,7 +103,7 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
     }
 
     /**
-     * Goodflow. Deleting a row, which has an empty rowKey value, should not throw an exception
+     * Deleting a row, which has an empty rowKey value, should throw an exception
      *
      * @throws IOException
      */
@@ -115,9 +115,12 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
         checkRows(false, false);
         final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
                 existingTable);
+
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Can't delete a row without an empty rowKey.");
+
         rowDeletionBuilder.where(HBaseDataContext.FIELD_ID).eq("");
         rowDeletionBuilder.execute();
-        checkRows(false, false);
     }
 
     /**


[18/31] metamodel git commit: Merge pull request #4 from GerardDellemann/feature/hbase-improvements-assume-in-tests

Posted by ka...@apache.org.
Merge pull request #4 from GerardDellemann/feature/hbase-improvements-assume-in-tests

Refactored to remove boiler plate isConfigured checks

Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/f48e9c83
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/f48e9c83
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/f48e9c83

Branch: refs/heads/master
Commit: f48e9c833f574663a29078ba67b87f6932685287
Parents: f4d2c97 717a3a4
Author: Gerard Dellemann <ge...@gmail.com>
Authored: Sun Jun 3 10:44:17 2018 +0200
Committer: GitHub <no...@github.com>
Committed: Sun Jun 3 10:44:17 2018 +0200

----------------------------------------------------------------------
 .../apache/metamodel/hbase/CreateTableTest.java | 218 ++++-----
 .../apache/metamodel/hbase/DeleteRowTest.java   | 212 ++++-----
 .../apache/metamodel/hbase/DropTableTest.java   |  27 +-
 .../metamodel/hbase/HBaseDataContextTest.java   |  17 +-
 .../apache/metamodel/hbase/HBaseTestCase.java   |  34 +-
 .../hbase/HBaseUpdateCallbackTest.java          |  57 ++-
 .../apache/metamodel/hbase/InsertRowTest.java   | 444 ++++++++-----------
 7 files changed, 423 insertions(+), 586 deletions(-)
----------------------------------------------------------------------



[03/31] metamodel git commit: Reverted changes to logic in core module and refactored HBaseColumn class so it now extends the AbstractColumn class (and implement the Column class).

Posted by ka...@apache.org.
Reverted changes to logic in core module and refactored HBaseColumn class so it now extends the AbstractColumn class (and implement the Column class).


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/2008c7eb
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/2008c7eb
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/2008c7eb

Branch: refs/heads/master
Commit: 2008c7eb641c647ffe2588a1119b5d52f8fc9809
Parents: f15d027
Author: arjansh <Ar...@humaninference.com>
Authored: Mon Apr 30 11:49:52 2018 +0200
Committer: arjansh <Ar...@humaninference.com>
Committed: Mon Apr 30 11:49:52 2018 +0200

----------------------------------------------------------------------
 .../metamodel/AbstractUpdateCallback.java       |   4 +-
 .../metamodel/data/AbstractRowBuilder.java      |  33 +-----
 .../insert/AbstractRowInsertionBuilder.java     |  10 +-
 hbase/pom.xml                                   | 111 ------------------
 .../org/apache/metamodel/hbase/HBaseColumn.java | 110 +++++++++++++++---
 .../hbase/HBaseRowInsertionBuilder.java         |   6 +-
 .../metamodel/hbase/HBaseUpdateCallback.java    |  24 +---
 .../hbase/qualifiers/FindQualifiersDriver.java  | 116 -------------------
 8 files changed, 110 insertions(+), 304 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/2008c7eb/core/src/main/java/org/apache/metamodel/AbstractUpdateCallback.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/metamodel/AbstractUpdateCallback.java b/core/src/main/java/org/apache/metamodel/AbstractUpdateCallback.java
index b41b481..4ff7279 100644
--- a/core/src/main/java/org/apache/metamodel/AbstractUpdateCallback.java
+++ b/core/src/main/java/org/apache/metamodel/AbstractUpdateCallback.java
@@ -117,7 +117,7 @@ public abstract class AbstractUpdateCallback implements UpdateCallback {
         return update(getTable(tableName));
     }
 
-    protected Table getTable(String tableName) {
+    private Table getTable(String tableName) {
         Table table = getDataContext().getTableByQualifiedLabel(tableName);
         if (table == null) {
             throw new IllegalArgumentException("No such table: " + tableName);
@@ -159,7 +159,7 @@ public abstract class AbstractUpdateCallback implements UpdateCallback {
             UnsupportedOperationException {
         return new DeleteAndInsertBuilder(this, table);
     }
-
+    
     public UpdateSummary getUpdateSummary() {
         return DefaultUpdateSummary.unknownUpdates();
     }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/2008c7eb/core/src/main/java/org/apache/metamodel/data/AbstractRowBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/metamodel/data/AbstractRowBuilder.java b/core/src/main/java/org/apache/metamodel/data/AbstractRowBuilder.java
index 4fdbc59..144017c 100644
--- a/core/src/main/java/org/apache/metamodel/data/AbstractRowBuilder.java
+++ b/core/src/main/java/org/apache/metamodel/data/AbstractRowBuilder.java
@@ -40,10 +40,6 @@ public abstract class AbstractRowBuilder<RB extends RowBuilder<?>> implements Ro
         this(table.getColumns());
     }
 
-    public AbstractRowBuilder(Table table, int numberOfValues) {
-        this(table.getColumns(), numberOfValues);
-    }
-
     public AbstractRowBuilder(List<Column> columns) {
         _columns = columns.toArray(new Column[columns.size()]);
         _explicitNulls = new boolean[_columns.length];
@@ -51,14 +47,6 @@ public abstract class AbstractRowBuilder<RB extends RowBuilder<?>> implements Ro
         _styles = new Style[_columns.length];
     }
 
-    public AbstractRowBuilder(List<Column> columns, int numberOfValues) {
-        _columns = columns.toArray(new Column[columns.size()]);
-        _explicitNulls = new boolean[numberOfValues];
-        _values = new Object[numberOfValues];
-        _styles = new Style[numberOfValues];
-        setColumns(columns);
-    }
-
     /**
      * Gets a boolean array indicating if any of the values have been explicitly
      * set to null (as opposed to just not set)
@@ -83,8 +71,7 @@ public abstract class AbstractRowBuilder<RB extends RowBuilder<?>> implements Ro
 
     @Override
     public final Row toRow() {
-        return new DefaultRow(new SimpleDataSetHeader(Arrays.stream(_columns).map(SelectItem::new).collect(Collectors
-                .toList())), _values);
+        return new DefaultRow(new SimpleDataSetHeader(Arrays.stream(_columns).map(SelectItem::new).collect(Collectors.toList())), _values);
     }
 
     @Override
@@ -159,22 +146,4 @@ public abstract class AbstractRowBuilder<RB extends RowBuilder<?>> implements Ro
         }
         return false;
     }
-
-    public void setColumns(List<Column> columns) {
-        if (columns.size() != _columns.length) {
-            throw new IllegalArgumentException("The amount of columns don't match");
-        }
-        for (int i = 0; i < _columns.length; i++) {
-            _columns[i] = columns.get(i);
-        }
-    }
-
-    public void setValues(Object[] values) {
-        if (values.length != _values.length) {
-            throw new IllegalArgumentException("The amount of values don't match");
-        }
-        for (int i = 0; i < values.length; i++) {
-            _values[i] = values[i];
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/2008c7eb/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java b/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
index 185f942..58c0da8 100644
--- a/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
+++ b/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
@@ -18,8 +18,6 @@
  */
 package org.apache.metamodel.insert;
 
-import java.util.List;
-
 import org.apache.metamodel.UpdateCallback;
 import org.apache.metamodel.data.AbstractRowBuilder;
 import org.apache.metamodel.data.Row;
@@ -27,6 +25,8 @@ import org.apache.metamodel.query.SelectItem;
 import org.apache.metamodel.schema.Column;
 import org.apache.metamodel.schema.Table;
 
+import java.util.List;
+
 /**
  * Abstract implementation of the {@link RowInsertionBuilder} interface,
  * provided as a convenience to {@link RowInsertable} implementations. Handles
@@ -44,12 +44,6 @@ public abstract class AbstractRowInsertionBuilder<U extends UpdateCallback> exte
         _table = table;
     }
 
-    public AbstractRowInsertionBuilder(U updateCallback, Table table, int numberOfValues) {
-        super(table, numberOfValues);
-        _updateCallback = updateCallback;
-        _table = table;
-    }
-
     @Override
     public Table getTable() {
         return _table;

http://git-wip-us.apache.org/repos/asf/metamodel/blob/2008c7eb/hbase/pom.xml
----------------------------------------------------------------------
diff --git a/hbase/pom.xml b/hbase/pom.xml
index 47101ba..3a1febe 100644
--- a/hbase/pom.xml
+++ b/hbase/pom.xml
@@ -118,113 +118,6 @@
 					<groupId>commons-httpclient</groupId>
 					<artifactId>commons-httpclient</artifactId>
 				</exclusion>
-				<!-- <exclusion>
-					<groupId>org.codehaus.jackson</groupId>
-					<artifactId>jackson-mapper-asl</artifactId>
-				</exclusion> -->
-				<exclusion>
-					<groupId>org.codehaus.jackson</groupId>
-					<artifactId>jackson-core</artifactId>
-				</exclusion>
-				<!-- <exclusion>
-					<groupId>org.codehaus.jackson</groupId>
-					<artifactId>jackson-core-asl</artifactId>
-				</exclusion> -->
-				<exclusion>
-					<groupId>org.codehaus.jackson</groupId>
-					<artifactId>jackson-jaxrs</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.codehaus.jackson</groupId>
-					<artifactId>jackson-xc</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-				<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-server</artifactId>
-			<version>${hbase.version}</version>
-			<exclusions>
-				<exclusion>
-					<groupId>jdk.tools</groupId>
-					<artifactId>jdk.tools</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<exclusion>
-					<artifactId>log4j</artifactId>
-					<groupId>log4j</groupId>
-				</exclusion>
-				<exclusion>
-					<artifactId>commons-logging</artifactId>
-					<groupId>commons-logging</groupId>
-				</exclusion>
-				<exclusion>
-					<artifactId>jetty</artifactId>
-					<groupId>org.mortbay.jetty</groupId>
-				</exclusion>
-				<exclusion>
-					<artifactId>jetty-util</artifactId>
-					<groupId>org.mortbay.jetty</groupId>
-				</exclusion>
-				<exclusion>
-						<groupId>com.github.stephenc.findbugs</groupId>
-						<artifactId>findbugs-annotations</artifactId>
-					</exclusion>
-				<exclusion>
-					<groupId>tomcat</groupId>
-					<artifactId>jasper-runtime</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>tomcat</groupId>
-					<artifactId>jasper-compiler</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>javax.servlet</groupId>
-					<artifactId>servlet-api</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>google-collections</groupId>
-					<artifactId>google-collections</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>net.sourceforge.collections</groupId>
-					<artifactId>collections-generic</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.sun.jersey</groupId>
-					<artifactId>jersey-core</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.sun.jersey</groupId>
-					<artifactId>jersey-server</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jsp-2.1</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.sun.jersey</groupId>
-					<artifactId>jersey-json</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jsp-api-2.1</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>io.netty</groupId>
-					<artifactId>netty</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>io.netty</groupId>
-					<artifactId>netty-all</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>commons-httpclient</groupId>
-					<artifactId>commons-httpclient</artifactId>
-				</exclusion>
 				<exclusion>
 					<groupId>org.codehaus.jackson</groupId>
 					<artifactId>jackson-mapper-asl</artifactId>
@@ -245,10 +138,6 @@
 					<groupId>org.codehaus.jackson</groupId>
 					<artifactId>jackson-xc</artifactId>
 				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>servlet-api-2.5</artifactId>
-				</exclusion>
 			</exclusions>
 		</dependency>
 		<dependency>

http://git-wip-us.apache.org/repos/asf/metamodel/blob/2008c7eb/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
index abe37d1..067b59f 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
@@ -18,33 +18,117 @@
  */
 package org.apache.metamodel.hbase;
 
-public final class HBaseColumn {
-    private String columnFamily;
-    private String qualifier;
+import org.apache.metamodel.schema.AbstractColumn;
+import org.apache.metamodel.schema.ColumnType;
+import org.apache.metamodel.schema.ColumnTypeImpl;
+import org.apache.metamodel.schema.SuperColumnType;
+import org.apache.metamodel.schema.Table;
 
-    public HBaseColumn() {
-        columnFamily = "";
-        qualifier = "";
+public final class HBaseColumn extends AbstractColumn {
+    private final String columnFamily;
+    private final String qualifier;
+    private final Table table;
+    private final boolean primaryKey;
+    private final ColumnType columnType;
+    private final int columnNumber;
+
+    public HBaseColumn(String columnFamily, Table table) {
+        this(columnFamily, null, table, -1);
+    }
+
+    public HBaseColumn(String columnFamily, String qualifier, Table table) {
+        this(columnFamily, qualifier, table, -1);
+    }
+
+    public HBaseColumn(String columnFamily, Table table, int columnNumber) {
+        this(columnFamily, null, table, columnNumber);
     }
 
-    public HBaseColumn(String columnFamily, String qualifier) {
+    public HBaseColumn(String columnFamily, String qualifier, Table table, int columnNumber) {
+        if (columnFamily == null) {
+            throw new IllegalArgumentException("Column family isn't allowed to be null.");
+        } else if (table == null) {
+            throw new IllegalArgumentException("Table isn't allowed to be null.");
+        }
+        
         this.columnFamily = columnFamily;
         this.qualifier = qualifier;
+        this.table = table;
+        this.columnNumber = columnNumber;
+
+        primaryKey = HBaseDataContext.FIELD_ID.equals(columnFamily);
+        
+        if (primaryKey || qualifier != null) {
+            columnType = new ColumnTypeImpl("BYTE[]", SuperColumnType.LITERAL_TYPE);
+        } else {
+            columnType = ColumnType.LIST;
+        }
     }
 
     public String getColumnFamily() {
         return columnFamily;
     }
 
-    public void setColumnFamily(String columnFamily) {
-        this.columnFamily = columnFamily;
-    }
-
     public String getQualifier() {
         return qualifier;
     }
 
-    public void setQualifier(String qualifier) {
-        this.qualifier = qualifier;
+    @Override
+    public String getName() {
+        if (qualifier == null) {
+            return columnFamily;
+        }
+        return columnFamily + ":" + qualifier;
+    }
+
+    @Override
+    public int getColumnNumber() {
+        return columnNumber;
+    }
+
+    @Override
+    public ColumnType getType() {
+        return columnType;
+    }
+
+    @Override
+    public Table getTable() {
+        return table;
+    }
+
+    @Override
+    public Boolean isNullable() {
+        return !primaryKey;
+    }
+
+    @Override
+    public String getRemarks() {
+        return null;
+    }
+
+    @Override
+    public Integer getColumnSize() {
+        return null;
+    }
+
+    @Override
+    public String getNativeType() {
+        // TODO: maybe change if no qualifier is present (and not identifier column).
+        return "byte[]";
+    }
+
+    @Override
+    public boolean isIndexed() {
+        return false;
+    }
+
+    @Override
+    public boolean isPrimaryKey() {
+        return primaryKey;
+    }
+
+    @Override
+    public String getQuote() {
+        return null;
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/2008c7eb/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
index 358d743..d77cd04 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
@@ -26,8 +26,8 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
 
     private final HBaseColumn[] _outputColumns;
 
-    public HBaseRowInsertionBuilder(HBaseUpdateCallback updateCallback, Table table, HBaseColumn[] outputColumns) {
-        super(updateCallback, table, outputColumns.length);
+    public HBaseRowInsertionBuilder(HBaseUpdateCallback updateCallback, HBaseTable table, HBaseColumn[] outputColumns) {
+        super(updateCallback, table.setColumns(outputColumns));
         _outputColumns = outputColumns;
     }
 
@@ -44,7 +44,7 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
                 int indexOfTablesColumn = 0;
 
                 while (!matchingColumnFound && indexOfTablesColumn < table.getColumnCount()) {
-                    if (outputColumns[i].getColumnFamily().equals(table.getColumn(indexOfTablesColumn).getName())) {
+                    if (outputColumns[i].equals(table.getColumn(indexOfTablesColumn))) {
                         matchingColumnFound = true;
                     } else {
                         indexOfTablesColumn++;

http://git-wip-us.apache.org/repos/asf/metamodel/blob/2008c7eb/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
index 271aa87..6cc1522 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
@@ -82,22 +82,12 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
         throw new UnsupportedOperationException("Use insertInto(String tableName, HBaseColumn[] outputColumns)");
     }
 
-    public HBaseRowInsertionBuilder insertInto(String tableName, HBaseColumn[] outputColumns)
-            throws IllegalArgumentException, IllegalStateException, UnsupportedOperationException {
-        Table table = getTable(tableName);
-        return insertInto(table, outputColumns);
-    }
-
-    public HBaseRowInsertionBuilder insertInto(Table table, HBaseColumn[] outputColumns)
-            throws IllegalArgumentException, IllegalStateException, UnsupportedOperationException {
-        validateTable(table);
-        return new HBaseRowInsertionBuilder(this, table, outputColumns);
-    }
-
-    private void validateTable(Table table) {
-        if (!(table instanceof HBaseTable)) {
-            throw new IllegalArgumentException("Not a valid HBase table: " + table);
+    public HBaseRowInsertionBuilder insertInto(Table table, HBaseColumn[] columns)
+            throws IllegalArgumentException {
+        if (table instanceof HBaseTable) {
+            return new HBaseRowInsertionBuilder(this, (HBaseTable) table, columns);
         }
+        throw new IllegalArgumentException("Not an HBase table: " + table);
     }
 
     protected synchronized void writeRow(HBaseTable hBaseTable, HBaseColumn[] outputColumns, Object[] values) {
@@ -120,10 +110,6 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
         throw new UnsupportedOperationException();
     }
 
-    public boolean tableAlreadyExists(String tableName) {
-        return _dataContext.getMainSchema().getTableByName(tableName) == null ? false : true;
-    }
-
     public HBaseConfiguration getConfiguration() {
         return _configuration;
     }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/2008c7eb/hbase/src/main/java/org/apache/metamodel/hbase/qualifiers/FindQualifiersDriver.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/qualifiers/FindQualifiersDriver.java b/hbase/src/main/java/org/apache/metamodel/hbase/qualifiers/FindQualifiersDriver.java
deleted file mode 100644
index 9292d06..0000000
--- a/hbase/src/main/java/org/apache/metamodel/hbase/qualifiers/FindQualifiersDriver.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/**
- * 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.metamodel.hbase.qualifiers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
-import org.apache.hadoop.hbase.mapreduce.TableMapper;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.metamodel.hbase.HBaseConfiguration;
-
-public class FindQualifiersDriver extends Configured implements Tool {
-
-    static class OnlyColumnNameMapper extends TableMapper<Text, Text> {
-        @Override
-        protected void map(ImmutableBytesWritable key, Result value, final Context context) throws IOException,
-                InterruptedException {
-            CellScanner cellScanner = value.cellScanner();
-            while (cellScanner.advance()) {
-
-                Cell cell = cellScanner.current();
-                byte[] q = Bytes.copy(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
-
-                context.write(new Text(q), new Text());
-            }
-        }
-    }
-
-    static class OnlyColumnNameReducer extends Reducer<Text, Text, Text, Text> {
-
-        @Override
-        protected void reduce(Text key, Iterable<Text> values, Context context) throws IOException,
-                InterruptedException {
-            context.write(new Text(key), new Text());
-        }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Path outputPath = new Path("output/");
-        byte[] tableName = new String("ietsanders2").getBytes();
-        byte[] columnFamilyName = new String("data").getBytes();
-
-        Configuration configuration = createConfig();
-        FileSystem fileSystem = FileSystem.get(configuration);
-        fileSystem.delete(outputPath, true);
-
-        Job job = Job.getInstance(configuration, "Distinct_columns");
-        job.setJarByClass(this.getClass());
-
-        Scan scan = new Scan();
-        scan.setBatch(500);
-        scan.addFamily(columnFamilyName);
-        scan.setFilter(new KeyOnlyFilter()); // scan only key part of KeyValue (raw, column family, column)
-        scan.setCacheBlocks(false); // don't set to true for MR jobs
-
-        TextOutputFormat.setOutputPath(job, outputPath);
-
-        TableMapReduceUtil.initTableMapperJob(tableName, scan, OnlyColumnNameMapper.class, // mapper
-                Text.class, // mapper output key
-                Text.class, // mapper output value
-                job);
-
-        job.setNumReduceTasks(1);
-        job.setReducerClass(OnlyColumnNameReducer.class);
-
-        return job.waitForCompletion(true) ? 0 : 1;
-    }
-
-    protected Configuration createConfig() {
-        Configuration config = org.apache.hadoop.hbase.HBaseConfiguration.create();
-        config.set("hbase.zookeeper.quorum", "bigdatavm");
-        config.set("hbase.zookeeper.property.clientPort", Integer.toString(HBaseConfiguration.DEFAULT_ZOOKEEPER_PORT));
-        config.set("hbase.client.retries.number", Integer.toString(HBaseConfiguration.DEFAULT_HBASE_CLIENT_RETRIES));
-        config.set("zookeeper.session.timeout", Integer.toString(HBaseConfiguration.DEFAULT_ZOOKEEPER_SESSION_TIMEOUT));
-        config.set("zookeeper.recovery.retry", Integer.toString(HBaseConfiguration.DEFAULT_ZOOKEEPER_RECOVERY_RETRIES));
-        return config;
-    }
-
-    public static void main(String[] args) throws Exception {
-        int exitCode = ToolRunner.run(new FindQualifiersDriver(), args);
-        System.exit(exitCode);
-    }
-}


[15/31] metamodel git commit: HBase improvements (small refactorings, add comments)

Posted by ka...@apache.org.
HBase improvements (small refactorings, add comments)


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/cd755a1c
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/cd755a1c
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/cd755a1c

Branch: refs/heads/master
Commit: cd755a1cf5ea3762463c577dd220162dee14dfd9
Parents: 703fb40
Author: Gerard Dellemann <g....@quadient.com>
Authored: Thu May 31 14:10:13 2018 +0200
Committer: Gerard Dellemann <g....@quadient.com>
Committed: Thu May 31 14:10:13 2018 +0200

----------------------------------------------------------------------
 .../org/apache/metamodel/hbase/HBaseClient.java | 23 ++++---
 .../hbase/HBaseRowInsertionBuilder.java         |  7 +--
 .../org/apache/metamodel/hbase/HBaseTable.java  |  2 +-
 .../apache/metamodel/hbase/CreateTableTest.java |  8 +--
 .../apache/metamodel/hbase/DeleteRowTest.java   | 42 ++++++++++---
 .../apache/metamodel/hbase/DropTableTest.java   |  2 +-
 .../apache/metamodel/hbase/HBaseTestCase.java   | 10 ++-
 .../hbase/HBaseUpdateCallbackTest.java          | 64 ++++++++++++++++++--
 .../apache/metamodel/hbase/InsertRowTest.java   | 44 +++++++-------
 9 files changed, 147 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/cd755a1c/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
index 979322e..8eda465 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
@@ -93,14 +93,19 @@ public final class HBaseClient {
         if (tableName == null || rowKey == null) {
             throw new IllegalArgumentException("Can't delete a row without having tableName or rowKey");
         }
-        try (final Table table = _connection.getTable(TableName.valueOf(tableName));) {
-            if (rowExists(table, rowKey) == true) {
-                table.delete(new Delete(Bytes.toBytes(rowKey.toString())));
-            } else {
-                logger.warn("Rowkey with value " + rowKey.toString() + " doesn't exist in the table");
+        byte[] rowKeyAsByteArray = Bytes.toBytes(rowKey.toString());
+        if (rowKeyAsByteArray.length > 0) {
+            try (final Table table = _connection.getTable(TableName.valueOf(tableName));) {
+                if (rowExists(table, rowKeyAsByteArray) == true) {
+                    table.delete(new Delete(rowKeyAsByteArray));
+                } else {
+                    logger.warn("Rowkey with value " + rowKey.toString() + " doesn't exist in the table");
+                }
+            } catch (IOException e) {
+                throw new MetaModelException(e);
             }
-        } catch (IOException e) {
-            throw new MetaModelException(e);
+        } else {
+            logger.info("Have not deleted a row, which has an empty (\"\") rowKey.");
         }
     }
 
@@ -111,8 +116,8 @@ public final class HBaseClient {
      * @return boolean
      * @throws IOException
      */
-    private boolean rowExists(Table table, Object rowKey) throws IOException {
-        final Get get = new Get(Bytes.toBytes(rowKey.toString()));
+    private boolean rowExists(Table table, byte[] rowKey) throws IOException {
+        final Get get = new Get(rowKey);
         return !table.get(get).isEmpty();
     }
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/cd755a1c/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
index 9091788..4ae97c1 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
@@ -63,13 +63,12 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
     }
 
     /**
-     * Returns the index of the ID-column (see {@link HBaseDataContext#FIELD_ID}) in an array of HBaseColumns. When no
-     * ID-column is found, then null is returned.
+     * Returns the index of the ID-column (see {@link HBaseDataContext#FIELD_ID}) in an array of HBaseColumns.
      *
      * @param columns
-     * @return {@link Integer}
+     * @return index of the ID-column
      */
-    private static Integer getIndexOfIdColumn(final List<HBaseColumn> columns) {
+    private static int getIndexOfIdColumn(final List<HBaseColumn> columns) {
         for (int i = 0; i < columns.size(); i++) {
             if (HBaseDataContext.FIELD_ID.equals(columns.get(i).getColumnFamily())) {
                 return i;

http://git-wip-us.apache.org/repos/asf/metamodel/blob/cd755a1c/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
index 0dd2558..c080856 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
@@ -151,7 +151,7 @@ final class HBaseTable extends MutableTable {
     /**
      * Returns the column families for this HBase table.
      *
-     * @return {@link Set}
+     * @return {@link Set}<{@link String}> of columnFamilies
      */
     Set<String> getColumnFamilies() {
         return getColumnsInternal()

http://git-wip-us.apache.org/repos/asf/metamodel/blob/cd755a1c/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
index 9ea04df..789b4bd 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
@@ -118,7 +118,7 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
         if (isConfigured()) {
             final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
             final LinkedHashMap<HBaseColumn, Object> row = createRow(table, null, CF_FOO, CF_BAR);
-            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromMap(row));
+            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
             try {
                 final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
                         .createTable(getSchema(), TABLE_NAME);
@@ -196,7 +196,7 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
         if (isConfigured()) {
             final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
             final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
-            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromMap(row));
+            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
             try {
                 final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
                         .createTable(getSchema(), TABLE_NAME);
@@ -220,7 +220,7 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
         if (isConfigured()) {
             final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
             final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
-            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromMap(row));
+            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
             try {
                 getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
                 checkSuccesfullyInsertedTable();
@@ -237,7 +237,7 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      * Creates a set of columnFamilies out of a list of hbaseColumns
      *
      * @param columns
-     * @return {@link LinkedHashSet}
+     * @return {@link Set}<{@link String}> of columnFamilies
      */
     private static Set<String> getColumnFamilies(List<HBaseColumn> columns) {
         return columns.stream().map(HBaseColumn::getColumnFamily).distinct().collect(Collectors.toSet());

http://git-wip-us.apache.org/repos/asf/metamodel/blob/cd755a1c/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
index 893b956..ef594c8 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
@@ -64,8 +64,8 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
     public void testHBaseClientNullAtBuilder() throws IOException {
         if (isConfigured()) {
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID,
+                        CF_FOO, CF_BAR);
                 new HBaseRowDeletionBuilder(null, existingTable);
                 fail("Should get an exception that hBaseClient can't be null.");
             } catch (IllegalArgumentException e) {
@@ -84,8 +84,8 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
     public void testNotSettingRowkey() throws IOException {
         if (isConfigured()) {
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID,
+                        CF_FOO, CF_BAR);
                 getUpdateCallback().deleteFrom(existingTable).execute();
                 fail("Should get an exception that the columnFamily doesn't exist.");
             } catch (MetaModelException e) {
@@ -137,8 +137,8 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
     public void testDeletingNotExistingRow() {
         if (isConfigured()) {
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID,
+                        CF_FOO, CF_BAR);
 
                 checkRows(false);
                 final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback()
@@ -156,16 +156,40 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
     }
 
     /**
+     * Goodflow. Deleting a row, which has an empty rowKey value, should not throw an exception
+     */
+    public void testUsingAnEmptyRowKeyValue() {
+        if (isConfigured()) {
+            try {
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID,
+                        CF_FOO, CF_BAR);
+
+                checkRows(false);
+                final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback()
+                        .deleteFrom(existingTable);
+                rowDeletionBuilder.setKey("");
+                rowDeletionBuilder.execute();
+                checkRows(false);
+            } catch (Exception e) {
+                fail("Should not get an exception that the rowkey is empty.");
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
      * Goodflow. Deleting a row succesfully.
      */
     public void testDeleteRowSuccesfully() {
         if (isConfigured()) {
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID,
+                        CF_FOO, CF_BAR);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
                         CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
 
                 checkRows(false);
                 final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable,

http://git-wip-us.apache.org/repos/asf/metamodel/blob/cd755a1c/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
index 39bb06d..afcc4eb 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
@@ -80,7 +80,7 @@ public class DropTableTest extends HBaseUpdateCallbackTest {
     public void testDropTableSuccesfully() throws IOException {
         if (isConfigured()) {
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
                 getUpdateCallback().dropTable(existingTable).execute();
                 try (final Admin admin = getDataContext().getAdmin()) {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/cd755a1c/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
index 70fd2e3..5f494b7 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
@@ -28,6 +28,10 @@ import org.junit.AfterClass;
 
 import junit.framework.TestCase;
 
+/**
+ * Properly configure before executing these tests. 
+ * See the {@link HBaseTestCase#setUp()} and {@link HBaseTestCase#getPropertyFilePath()} methods.
+ */
 public abstract class HBaseTestCase extends TestCase {
 
     // TableName
@@ -93,7 +97,11 @@ public abstract class HBaseTestCase extends TestCase {
         _dataContext.getConnection().close();
     }
 
-    private String getPropertyFilePath() {
+    /**
+     * Get's the test configuration file. An example file can be found at the root folder of this project.
+     * @return Location of the configuration file.
+     */
+    protected String getPropertyFilePath() {
         String userHome = System.getProperty("user.home");
         return userHome + "/metamodel-integrationtest-configuration.properties";
     }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/cd755a1c/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
index 60a5ba6..5085689 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
@@ -68,6 +68,10 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         super.tearDown();
     }
 
+    /**
+     * Drop the table if it exists. 
+     * After that check in the schema and the datastore if the actions have been executed succesfully.
+     */
     protected void dropTableIfItExists() {
         final Table table = schema.getTableByName(TABLE_NAME);
         if (table != null) {
@@ -83,6 +87,11 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         }
     }
 
+    /**
+     * Check if the table has been inserted succesfully.
+     * Checks are performed in the schema and the datastore.
+     * @throws IOException because the admin object needs to be created
+     */
     protected void checkSuccesfullyInsertedTable() throws IOException {
         // Check the schema
         assertNotNull(schema.getTableByName(TABLE_NAME));
@@ -94,8 +103,17 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         }
     }
 
-    protected HBaseTable createAndInsertTable(final String tableName, final String idColumn, final String columnFamily1,
-            final String columnFamily2) throws IOException {
+    /**
+     * Create a test HBase table and add it to the datastore
+     * @param tableName name of the table
+     * @param idColumn required ID-column
+     * @param columnFamily1 required columnFamily 1
+     * @param columnFamily2 required columnFamily 2
+     * @return created and add {@link HBaseTable}
+     * @throws IOException
+     */
+    protected HBaseTable createAndAddTableToDatastore(final String tableName, final String idColumn,
+            final String columnFamily1, final String columnFamily2) throws IOException {
         final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
         columnFamilies.add(idColumn);
         columnFamilies.add(columnFamily1);
@@ -105,6 +123,15 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         return (HBaseTable) getDataContext().getDefaultSchema().getTableByName(tableName);
     }
 
+    /**
+     * Create a test HBase table
+     * @param tableName name of the table
+     * @param idColumn required ID-column
+     * @param columnFamily1 required columnFamily 1
+     * @param columnFamily2 required columnFamily 2
+     * @param columnFamily3 columnFamily 3 is not required and can be used to test errors
+     * @return created {@link HBaseTable}
+     */
     protected HBaseTable createHBaseTable(final String tableName, final String idColumn, final String columnFamily1,
             final String columnFamily2, final String columnFamily3) {
         String[] columnNames;
@@ -121,6 +148,14 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         return new HBaseTable(getDataContext(), tableDef, schema, ColumnType.STRING);
     }
 
+    /**
+     * Creates a map that represents a row
+     * @param table HBaseTable
+     * @param idColumn required ID-column
+     * @param columnFamily1 required columnFamily 1
+     * @param columnFamily2 required columnFamily 1
+     * @return {@link LinkedHashMap}<{@link HBaseColumn}, {@link Object}>
+     */
     protected static LinkedHashMap<HBaseColumn, Object> createRow(final HBaseTable table, final String idColumn,
             final String columnFamily1, final String columnFamily2) {
         final LinkedHashMap<HBaseColumn, Object> map = new LinkedHashMap<>();
@@ -153,12 +188,22 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         return map;
     }
 
-    protected static List<HBaseColumn> getHBaseColumnsFromMap(final LinkedHashMap<HBaseColumn, Object> map) {
+    /**
+     * Get the HBaseColumns out of a mapped row
+     * @param row {@link LinkedHashMap}<{@link HBaseColumn}, {@link Object}>
+     * @return {@link List}<{@link HBaseColumn}>
+     */
+    protected static List<HBaseColumn> getHBaseColumnsFromRow(final LinkedHashMap<HBaseColumn, Object> row) {
         final List<HBaseColumn> columns = new ArrayList<>();
-        columns.addAll(map.keySet());
+        columns.addAll(row.keySet());
         return columns;
     }
 
+    /**
+     * Set the values of a {@link HBaseRowInsertionBuilder} from the values in the mapped row
+     * @param row {@link LinkedHashMap}<{@link HBaseColumn}, {@link Object}> containing the values
+     * @param rowInsertionBuilder insertionBuilder to be set
+     */
     protected void setValuesInInsertionBuilder(final LinkedHashMap<HBaseColumn, Object> row,
             final HBaseRowInsertionBuilder rowInsertionBuilder) {
         int i = 0;
@@ -168,6 +213,11 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         }
     }
 
+    /**
+     * Checks that the table does or doesn't have rows, depending on the rowsExists parameter
+     * @param rowsExist true, check that the rows exists. false, check that the result is empty.
+     * @throws IOException
+     */
     protected void checkRows(final boolean rowsExist) throws IOException {
         try (org.apache.hadoop.hbase.client.Table table = getDataContext().getConnection().getTable(TableName.valueOf(
                 TABLE_NAME))) {
@@ -186,6 +236,12 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         }
     }
 
+    /**
+     * Warn that the test(method) of a class is not executed, because the test-file hasn't been set.
+     * See {@link HBaseTestCase#getPropertyFilePath} 
+     * @param className 
+     * @param methodName
+     */
     protected void warnAboutANotExecutedTest(String className, String methodName) {
         String logWarning = "Test \"" + className + "#" + methodName
                 + "()\" is not executed, because the HBasetest is not configured.";

http://git-wip-us.apache.org/repos/asf/metamodel/blob/cd755a1c/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
index 1a9a424..db04527 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
@@ -48,7 +48,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     public void testOnlyUsingTableParameter() throws IOException {
         if (isConfigured()) {
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
                 getUpdateCallback().insertInto(existingTable);
                 fail("Should get an exception that this method is not supported");
@@ -69,11 +69,11 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
         if (isConfigured()) {
             final MutableTable mutableTable = new MutableTable();
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
                         CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
                 getUpdateCallback().insertInto(mutableTable, columns);
                 fail("Should get an exception that the type of the table is wrong.");
             } catch (IllegalArgumentException e) {
@@ -92,7 +92,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     public void testColumnsNullAtUpdateCallBack() throws IOException {
         if (isConfigured()) {
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
                 getUpdateCallback().insertInto(existingTable, null);
                 fail("Should get an exception that the columns list is null.");
@@ -112,7 +112,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     public void testColumnsEmptyAtUpdateCallBack() throws IOException {
         if (isConfigured()) {
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
                 getUpdateCallback().insertInto(existingTable, new ArrayList<HBaseColumn>());
                 fail("Should get an exception that the columns list is empty.");
@@ -132,7 +132,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     public void testColumnsEmptyAtBuilder() throws IOException {
         if (isConfigured()) {
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
                 List<HBaseColumn> emptyList = new ArrayList<>();
                 new HBaseRowInsertionBuilder(getUpdateCallback(), existingTable, emptyList);
@@ -155,11 +155,11 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
             final HBaseTable wrongTable = createHBaseTable("NewTableNotInSchema", HBaseDataContext.FIELD_ID, "cf1",
                     "cf2", null);
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
                         CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
                 getUpdateCallback().insertInto(wrongTable, columns);
                 fail("Should get an exception that the table isn't in the schema.");
             } catch (MetaModelException e) {
@@ -179,10 +179,10 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     public void testIDColumnDoesntExistInColumnsArray() throws IOException {
         if (isConfigured()) {
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, null, CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
                 getUpdateCallback().insertInto(existingTable, columns);
                 fail("Should get an exception that ID-column doesn't exist.");
             } catch (MetaModelException e) {
@@ -202,11 +202,11 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
         if (isConfigured()) {
             final String wrongColumnFamily = "wrongColumnFamily";
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
                         CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
                 final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         wrongColumnFamily, null);
                 getUpdateCallback().insertInto(wrongTable, columns);
@@ -229,11 +229,11 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
         if (isConfigured()) {
             final String wrongColumnFamily = "newColumnFamily";
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
                         CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
                 final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
                         wrongColumnFamily);
                 getUpdateCallback().insertInto(wrongTable, columns);
@@ -257,7 +257,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
                 final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
-                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
+                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
                 final Object[] values = new String[] { "Values" };
                 new HBaseClient(getDataContext().getConnection()).insertRow(null, columns, values, 0);
                 fail("Should get an exception that tableName is null");
@@ -301,7 +301,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
                 final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
-                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
+                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
                 new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, null, 0);
                 fail("Should get an exception that values is null");
             } catch (IllegalArgumentException e) {
@@ -324,7 +324,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
                 final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
-                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
+                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
                 final Object[] values = new String[] { "Values" };
                 new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 10);
                 fail("Should get an exception that the indexOfIdColumn is incorrect");
@@ -348,7 +348,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
                 final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
-                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
+                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
                 final Object[] values = new String[] { null };
                 new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 0);
                 fail("Should get an exception that the indexOfIdColumn is incorrect");
@@ -369,11 +369,11 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     public void testInsertIntoWithoutExecute() {
         if (isConfigured()) {
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
                         CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
                 getUpdateCallback().insertInto(existingTable, columns);
             } catch (Exception e) {
                 fail("No exception should be thrown, when inserting into an existing table.");
@@ -416,11 +416,11 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     public void testInsertingSuccesfully() {
         if (isConfigured()) {
             try {
-                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
                         CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
 
                 checkRows(false);
                 final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable,


[07/31] metamodel git commit: HBase improvements: refactoring, new unittests

Posted by ka...@apache.org.
HBase improvements: refactoring, new unittests


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/3ada7eb4
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/3ada7eb4
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/3ada7eb4

Branch: refs/heads/master
Commit: 3ada7eb48e0599b507024b1b6c10be3b7cceb63f
Parents: 38d8bc3
Author: Gerard Dellemann <g....@quadient.com>
Authored: Thu May 24 16:12:20 2018 +0200
Committer: Gerard Dellemann <g....@quadient.com>
Committed: Thu May 24 16:12:20 2018 +0200

----------------------------------------------------------------------
 .../org/apache/metamodel/hbase/HBaseClient.java | 164 ++++++++++++++
 .../org/apache/metamodel/hbase/HBaseColumn.java |  28 ++-
 .../hbase/HBaseCreateTableBuilder.java          |  94 ++++++--
 .../hbase/HBaseRowDeletionBuilder.java          |  24 ++-
 .../hbase/HBaseRowInsertionBuilder.java         |  89 ++++----
 .../org/apache/metamodel/hbase/HBaseTable.java  |  90 ++++++--
 .../metamodel/hbase/HBaseTableDropBuilder.java  |  25 ++-
 .../metamodel/hbase/HBaseUpdateCallback.java    |  99 +++------
 .../org/apache/metamodel/hbase/HBaseWriter.java | 147 -------------
 .../metamodel/hbase/HBaseDataContextTest.java   |  98 +++------
 .../apache/metamodel/hbase/HBaseTestCase.java   |  58 ++++-
 .../hbase/HBaseUpdateCallbackTest.java          | 213 +++++++++++++++++++
 12 files changed, 742 insertions(+), 387 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/3ada7eb4/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
new file mode 100644
index 0000000..1e957fc
--- /dev/null
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
@@ -0,0 +1,164 @@
+/**
+ * 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.metamodel.hbase;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metamodel.MetaModelException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class can perform client-operations on a HBase datastore
+ */
+public final class HBaseClient {
+
+    private static final Logger logger = LoggerFactory.getLogger(HBaseClient.class);
+
+    private final Connection _connection;
+
+    public HBaseClient(Connection connection) {
+        this._connection = connection;
+    }
+
+    /**
+     * Write a single row of values to a HBase table
+     * @param hBaseTable
+     * @param columns
+     * @param values
+     * @throws IOException
+     */
+    public void writeRow(HBaseTable hBaseTable, HBaseColumn[] columns, Object[] values) throws IOException {
+        try (final Table table = _connection.getTable(TableName.valueOf(hBaseTable.getName()))) {
+            int indexOfIdColumn = getIndexOfIdColumn(columns);
+
+            // Create a put with the values of indexOfIdColumn as rowkey
+            final Put put = new Put(Bytes.toBytes(values[indexOfIdColumn].toString()));
+
+            // Add the other values to the put
+            for (int i = 0; i < columns.length; i++) {
+                if (i != indexOfIdColumn) {
+                    put.addColumn(Bytes.toBytes(columns[i].getColumnFamily()), Bytes.toBytes(columns[i].getQualifier()),
+                            Bytes.toBytes(values[i].toString()));
+                }
+            }
+            // Add the put to the table
+            table.put(put);
+        }
+    }
+
+    /**
+     * Gets the index of the ID-column
+     * Throws an {@link MetaModelException} when no ID-column is found.
+     * @param columns
+     * @return 
+     */
+    private int getIndexOfIdColumn(HBaseColumn[] columns) {
+        int indexOfIdColumn = 0;
+        boolean idColumnFound = false;
+        while (!idColumnFound && indexOfIdColumn < columns.length) {
+            if (columns[indexOfIdColumn].getColumnFamily().equals(HBaseDataContext.FIELD_ID)) {
+                idColumnFound = true;
+            } else {
+                indexOfIdColumn++;
+            }
+        }
+        if (!idColumnFound) {
+            throw new MetaModelException("The ID Column family was not found");
+        }
+        return indexOfIdColumn;
+    }
+
+    /**
+     * Delete 1 row based on the key
+     * @param hBaseTable
+     * @param key
+     * @throws IOException
+     */
+    public void deleteRow(HBaseTable hBaseTable, Object key) throws IOException {
+        try (final Table table = _connection.getTable(TableName.valueOf(hBaseTable.getName()));) {
+            if (rowExists(table, key) == true) {
+                table.delete(new Delete(Bytes.toBytes(key.toString())));
+            } else {
+                logger.warn("Rowkey with value " + key.toString() + " doesn't exist in the table");
+            }
+        }
+    }
+
+    /**
+     * Checks in the HBase datastore if a row exists based on the key
+     * @param table
+     * @param key
+     * @return boolean
+     * @throws IOException
+     */
+    private boolean rowExists(Table table, Object key) throws IOException {
+        final Get get = new Get(Bytes.toBytes(key.toString()));
+        return !table.get(get).isEmpty();
+    }
+
+    /**
+     * Creates a HBase table based on a tableName and it's columnFamilies
+     * @param tableName
+     * @param columnFamilies
+     * @throws IOException
+     */
+    public void createTable(String tableName, Set<String> columnFamilies) throws IOException {
+        try (final Admin admin = _connection.getAdmin()) {
+            final TableName hBasetableName = TableName.valueOf(tableName);
+            final HTableDescriptor tableDescriptor = new HTableDescriptor(hBasetableName);
+            // Add all columnFamilies to the tableDescriptor.
+            for (final String columnFamilie : columnFamilies) {
+                // The ID-column isn't needed because, it will automatically be created.
+                if (!columnFamilie.equals(HBaseDataContext.FIELD_ID)) {
+                    tableDescriptor.addFamily(new HColumnDescriptor(columnFamilie));
+                }
+            }
+            admin.createTable(tableDescriptor);
+            final HTableDescriptor[] tables = admin.listTables();
+            if (tables.length != 1 && Bytes.equals(hBasetableName.getName(), tables[0].getTableName().getName())) {
+                throw new IOException("Failed create of table");
+            }
+        }
+    }
+
+    /**
+     * Disable and drop a table from a HBase datastore
+     * @param tableName
+     * @throws IOException
+     */
+    public void dropTable(String tableName) throws IOException {
+        try (final Admin admin = _connection.getAdmin()) {
+            final TableName hBasetableName = TableName.valueOf(tableName);
+            admin.disableTable(hBasetableName); // A table must be disabled first, before it can be deleted
+            admin.deleteTable(hBasetableName);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/metamodel/blob/3ada7eb4/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
index 067b59f..bd57c1c 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
@@ -18,6 +18,9 @@
  */
 package org.apache.metamodel.hbase;
 
+import java.util.LinkedHashSet;
+import java.util.Set;
+
 import org.apache.metamodel.schema.AbstractColumn;
 import org.apache.metamodel.schema.ColumnType;
 import org.apache.metamodel.schema.ColumnTypeImpl;
@@ -32,32 +35,32 @@ public final class HBaseColumn extends AbstractColumn {
     private final ColumnType columnType;
     private final int columnNumber;
 
-    public HBaseColumn(String columnFamily, Table table) {
+    public HBaseColumn(final String columnFamily, final Table table) {
         this(columnFamily, null, table, -1);
     }
 
-    public HBaseColumn(String columnFamily, String qualifier, Table table) {
+    public HBaseColumn(final String columnFamily, final String qualifier, final Table table) {
         this(columnFamily, qualifier, table, -1);
     }
 
-    public HBaseColumn(String columnFamily, Table table, int columnNumber) {
+    public HBaseColumn(final String columnFamily, final Table table, final int columnNumber) {
         this(columnFamily, null, table, columnNumber);
     }
 
-    public HBaseColumn(String columnFamily, String qualifier, Table table, int columnNumber) {
+    public HBaseColumn(final String columnFamily, final String qualifier, final Table table, final int columnNumber) {
         if (columnFamily == null) {
             throw new IllegalArgumentException("Column family isn't allowed to be null.");
         } else if (table == null) {
             throw new IllegalArgumentException("Table isn't allowed to be null.");
         }
-        
+
         this.columnFamily = columnFamily;
         this.qualifier = qualifier;
         this.table = table;
         this.columnNumber = columnNumber;
 
         primaryKey = HBaseDataContext.FIELD_ID.equals(columnFamily);
-        
+
         if (primaryKey || qualifier != null) {
             columnType = new ColumnTypeImpl("BYTE[]", SuperColumnType.LITERAL_TYPE);
         } else {
@@ -131,4 +134,17 @@ public final class HBaseColumn extends AbstractColumn {
     public String getQuote() {
         return null;
     }
+
+    /**
+     * Creates a set of columnFamilies out of an array of hbaseColumns
+     * @param hbaseColumns
+     * @return {@link LinkedHashSet}
+     */
+    public static Set<String> getColumnFamilies(HBaseColumn[] hbaseColumns) {
+        final LinkedHashSet<String> columnFamilies = new LinkedHashSet<String>();
+        for (int i = 0; i < hbaseColumns.length; i++) {
+            columnFamilies.add(hbaseColumns[i].getColumnFamily());
+        }
+        return columnFamilies;
+    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/3ada7eb4/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
index e1f1257..ffdc20a 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
@@ -19,53 +19,103 @@
 package org.apache.metamodel.hbase;
 
 import java.io.IOException;
-import java.util.LinkedHashSet;
+import java.util.Iterator;
 import java.util.Set;
 
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.create.AbstractTableCreationBuilder;
 import org.apache.metamodel.schema.MutableSchema;
-import org.apache.metamodel.schema.MutableTable;
 import org.apache.metamodel.schema.Schema;
 import org.apache.metamodel.schema.Table;
 import org.apache.metamodel.util.SimpleTableDef;
 
+/**
+ * A builder-class to create tables in a HBase datastore
+ */
 public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseUpdateCallback> {
 
-    private Set<String> columnFamilies;
+    private Set<String> _columnFamilies;
+
+    public HBaseCreateTableBuilder(HBaseUpdateCallback updateCallback, Schema schema, String name) {
+        this(updateCallback, schema, name, null);
+    }
 
+    /**
+     * Create a {@link HBaseCreateTableBuilder}.
+     * Throws an {@link IllegalArgumentException} if the schema isn't a {@link MutableSchema}.
+     * @param updateCallback
+     * @param schema
+     * @param name
+     * @param columnFamilies
+     */
     public HBaseCreateTableBuilder(HBaseUpdateCallback updateCallback, Schema schema, String name,
-            HBaseColumn[] outputColumns) {
+            Set<String> columnFamilies) {
         super(updateCallback, schema, name);
         if (!(schema instanceof MutableSchema)) {
-            throw new IllegalArgumentException("Not a valid schema: " + schema);
-        }
-        columnFamilies = new LinkedHashSet<String>();
-        for (int i = 0; i < outputColumns.length; i++) {
-            columnFamilies.add(outputColumns[i].getColumnFamily());
+            throw new IllegalArgumentException("Not a mutable schema: " + schema);
         }
+        this._columnFamilies = columnFamilies;
     }
 
     @Override
-    public Table execute() throws MetaModelException {
-        final MutableTable table = getTable();
-        final SimpleTableDef emptyTableDef = new SimpleTableDef(table.getName(), columnFamilies.toArray(
-                new String[columnFamilies.size()]));
+    public Table execute() {
+        checkColumnFamilies(_columnFamilies);
 
-        final HBaseUpdateCallback updateCallback = (HBaseUpdateCallback) getUpdateCallback();
+        final Table table = getTable();
 
+        // Add the table to the datastore
         try {
-            final HBaseWriter HbaseWriter = new HBaseWriter(HBaseDataContext.createConfig(updateCallback
-                    .getConfiguration()));
-            HbaseWriter.createTable(table.getName(), columnFamilies);
+            final HBaseClient hBaseClient = getUpdateCallback().getHBaseClient();
+            hBaseClient.createTable(table.getName(), _columnFamilies);
         } catch (IOException e) {
-            e.printStackTrace();
+            throw new MetaModelException(e);
         }
 
-        final MutableSchema schema = (MutableSchema) table.getSchema();
-        schema.addTable(new HBaseTable(updateCallback.getDataContext(), emptyTableDef, schema,
-                HBaseConfiguration.DEFAULT_ROW_KEY_TYPE));
-        return schema.getTableByName(table.getName());
+        // Update the schema
+        addNewTableToSchema(table);
+        return getSchema().getTableByName(table.getName());
     }
 
+    /**
+     * Check if the new table has columnFamilies and if the ID-column is included.
+     * Throws a {@link MetaModelException} if a check fails.
+     * @param columnFamilies
+     */
+    private void checkColumnFamilies(Set<String> columnFamilies) {
+        if (columnFamilies == null || columnFamilies.size() == 0) {
+            throw new MetaModelException("Creating a table without columnFamilies");
+        }
+        boolean idColumnFound = false;
+        final Iterator<String> iterator = columnFamilies.iterator();
+        while (!idColumnFound && iterator.hasNext()) {
+            if (iterator.next().equals(HBaseDataContext.FIELD_ID)) {
+                idColumnFound = true;
+            }
+        }
+        if (!idColumnFound) {
+            throw new MetaModelException("ColumnFamily: " + HBaseDataContext.FIELD_ID + " not found");
+        }
+    }
+
+    /**
+     * Set the columnFamilies
+     * @param columnFamilies
+     */
+    public void setColumnFamilies(Set<String> columnFamilies) {
+        this._columnFamilies = columnFamilies;
+    }
+
+    /**
+     * Add the new {@link Table} to the {@link MutableSchema}
+     * @param table
+     * @param updateCallback
+     * @return {@link MutableSchema}
+     */
+    private void addNewTableToSchema(final Table table) {
+        final MutableSchema schema = (MutableSchema) getSchema();
+        final SimpleTableDef emptyTableDef = new SimpleTableDef(table.getName(), _columnFamilies.toArray(
+                new String[_columnFamilies.size()]));
+        schema.addTable(new HBaseTable((HBaseDataContext) getUpdateCallback().getDataContext(), emptyTableDef, schema,
+                HBaseConfiguration.DEFAULT_ROW_KEY_TYPE));
+    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/3ada7eb4/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
index 925537a..3ba4ef8 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
@@ -18,30 +18,38 @@
  */
 package org.apache.metamodel.hbase;
 
+import java.io.IOException;
+
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.delete.AbstractRowDeletionBuilder;
 import org.apache.metamodel.schema.Table;
 
+/**
+ * A builder-class to delete rows in a HBase datastore
+ */
 public class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
 
-    private HBaseUpdateCallback _updateCallback;
+    private HBaseClient _hBaseClient;
     private Object _key;
 
-    public HBaseRowDeletionBuilder(HBaseUpdateCallback updateCallback, Table table) {
+    public HBaseRowDeletionBuilder(final HBaseClient hBaseWriter, final Table table) {
         super(table);
-        if (updateCallback == null) {
+        if (hBaseWriter == null) {
             throw new IllegalArgumentException("UpdateCallback cannot be null");
         }
-        _updateCallback = updateCallback;
-
+        this._hBaseClient = hBaseWriter;
     }
 
     @Override
-    public void execute() throws MetaModelException {
+    public synchronized void execute() {
         if (_key == null) {
-            throw new IllegalArgumentException("Key cannot be null");
+            throw new MetaModelException("Key cannot be null");
+        }
+        try {
+            _hBaseClient.deleteRow((HBaseTable) getTable(), _key);
+        } catch (IOException e) {
+            throw new MetaModelException(e);
         }
-        _updateCallback.deleteRow((HBaseTable) getTable(), _key);
     }
 
     public void setKey(Object _key) {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/3ada7eb4/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
index d77cd04..1c633f7 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
@@ -18,59 +18,70 @@
  */
 package org.apache.metamodel.hbase;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Set;
+
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.insert.AbstractRowInsertionBuilder;
-import org.apache.metamodel.schema.Table;
 
+/**
+ * A builder-class to insert rows in a HBase datastore
+ */
 public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
 
-    private final HBaseColumn[] _outputColumns;
+    private HBaseColumn[] _hbaseColumns;
 
-    public HBaseRowInsertionBuilder(HBaseUpdateCallback updateCallback, HBaseTable table, HBaseColumn[] outputColumns) {
-        super(updateCallback, table.setColumns(outputColumns));
-        _outputColumns = outputColumns;
-    }
-
-    @Override
-    public void execute() throws MetaModelException {
-        checkForMatchingColumnFamilies(getTable(), _outputColumns);
-        getUpdateCallback().writeRow((HBaseTable) getTable(), _outputColumns, getValues());
+    public HBaseRowInsertionBuilder(final HBaseUpdateCallback updateCallback, final HBaseTable table) {
+        super(updateCallback, table);
+        checkTable(updateCallback, table);
     }
 
-    private void checkForMatchingColumnFamilies(Table table, HBaseColumn[] outputColumns) {
-        for (int i = 0; i < outputColumns.length; i++) {
-            if (!outputColumns[i].getColumnFamily().equals(HBaseDataContext.FIELD_ID)) {
-                boolean matchingColumnFound = false;
-                int indexOfTablesColumn = 0;
-
-                while (!matchingColumnFound && indexOfTablesColumn < table.getColumnCount()) {
-                    if (outputColumns[i].equals(table.getColumn(indexOfTablesColumn))) {
-                        matchingColumnFound = true;
-                    } else {
-                        indexOfTablesColumn++;
-                    }
-                }
-
-                if (!matchingColumnFound) {
-                    throw new IllegalArgumentException(String.format(
-                            "OutputColumnFamily: %s doesn't exist in the schema of the table", outputColumns[i]
-                                    .getColumnFamily()));
-                }
-            }
+    /**
+     * Check if the table exits and it's columnFamilies exist
+     * If the table doesn't exist, then a {@link MetaModelException} is thrown
+     * @param updateCallback
+     * @param tableGettingInserts
+     */
+    private void checkTable(final HBaseUpdateCallback updateCallback, final HBaseTable tableGettingInserts) {
+        final HBaseTable tableInSchema = (HBaseTable) updateCallback.getDataContext().getDefaultSchema().getTableByName(
+                tableGettingInserts.getName());
+        if (tableInSchema == null) {
+            throw new MetaModelException("Trying to insert data into table: " + tableGettingInserts.getName()
+                    + ", which doesn't exist yet");
         }
+        tableInSchema.checkForNotMatchingColumns(tableGettingInserts.getColumnNames());
     }
 
-    public HBaseColumn[] getOutputColumns() {
-        return _outputColumns;
+    /**
+     * Set the hbaseColumns. Checks if the columnFamilies exist in the table.
+     * @param hbaseColumns a {@link IllegalArgumentException} is thrown when this parameter is null or empty
+     */
+    public void setHbaseColumns(HBaseColumn[] hbaseColumns) {
+        if (hbaseColumns == null || hbaseColumns.length == 0) {
+            throw new IllegalArgumentException("List of hbaseColumns is null or empty");
+        }
+        final Set<String> columnFamilies = HBaseColumn.getColumnFamilies(hbaseColumns);
+        final HBaseTable tableInSchema = (HBaseTable) getTable();
+        final ArrayList<String> columnFamiliesAsList = new ArrayList<String>();
+        columnFamiliesAsList.addAll(columnFamilies);
+        tableInSchema.checkForNotMatchingColumns(columnFamiliesAsList);
+        this._hbaseColumns = hbaseColumns;
     }
 
-    public void setOutputColumns(HBaseColumn[] outputColumns) {
-        if (outputColumns.length != _outputColumns.length) {
-            throw new IllegalArgumentException("The amount of outputColumns don't match");
+    @Override
+    public synchronized void execute() {
+        if (_hbaseColumns == null || _hbaseColumns.length == 0) {
+            throw new MetaModelException("The hbaseColumns-array is null or empty");
+        }
+        if (getValues() == null || getValues().length == 0) {
+            throw new MetaModelException("The values-array is null or empty");
         }
-        for (int i = 0; i < outputColumns.length; i++) {
-            _outputColumns[i] = outputColumns[i];
+        try {
+            final HBaseClient hBaseClient = getUpdateCallback().getHBaseClient();
+            hBaseClient.writeRow((HBaseTable) getTable(), _hbaseColumns, getValues());
+        } catch (IOException e) {
+            throw new MetaModelException(e);
         }
     }
-
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/3ada7eb4/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
index def7c9c..fee2f5d 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
@@ -43,49 +43,69 @@ final class HBaseTable extends MutableTable {
     private final transient HBaseDataContext _dataContext;
     private final transient ColumnType _defaultRowKeyColumnType;
 
+    /**
+     * Creates an HBaseTable. If the tableDef variable doesn't include the ID column (see {@link HBaseDataContext#FIELD_ID}). 
+     * Then it's first inserted.
+     * @param dataContext
+     * @param tableDef Table definition. The tableName, columnNames and columnTypes variables are used.
+     * @param schema {@link MutableSchema} where the table belongs to.
+     * @param defaultRowKeyColumnType This variable determines the {@link ColumnType}, 
+     * used when the tableDef doesn't include the ID column (see {@link HBaseDataContext#FIELD_ID}). 
+     */
     public HBaseTable(HBaseDataContext dataContext, SimpleTableDef tableDef, MutableSchema schema,
             ColumnType defaultRowKeyColumnType) {
         super(tableDef.getName(), TableType.TABLE, schema);
         _dataContext = dataContext;
         _defaultRowKeyColumnType = defaultRowKeyColumnType;
 
+        // Add the columns
         final String[] columnNames = tableDef.getColumnNames();
         if (columnNames == null || columnNames.length == 0) {
             logger.info("No user-defined columns specified for table {}. Columns will be auto-detected.");
         } else {
-
-            final ColumnType[] types = tableDef.getColumnTypes();
-            int columnNumber = 1;
-
-            for (int i = 0; i < columnNames.length; i++) {
-                String columnName = columnNames[i];
-                if (HBaseDataContext.FIELD_ID.equals(columnName)) {
-                    final ColumnType type = types[i];
-                    final MutableColumn idColumn = new MutableColumn(HBaseDataContext.FIELD_ID, type)
-                            .setPrimaryKey(true).setColumnNumber(columnNumber).setTable(this);
-                    addColumn(idColumn);
-                    columnNumber++;
+            final ColumnType[] columnTypes = tableDef.getColumnTypes();
+
+            // Find the ID-Column
+            boolean idColumnFound = false;
+            int indexOfIDColumn = 0;
+            while (!idColumnFound && indexOfIDColumn < columnNames.length) {
+                if (columnNames[indexOfIDColumn].equals(HBaseDataContext.FIELD_ID)) {
+                    idColumnFound = true;
+                } else {
+                    indexOfIDColumn++;
                 }
             }
 
-            if (columnNumber == 1) {
-                // insert a default definition of the id column
-                final MutableColumn idColumn = new MutableColumn(HBaseDataContext.FIELD_ID, defaultRowKeyColumnType)
-                        .setPrimaryKey(true).setColumnNumber(columnNumber).setTable(this);
-                addColumn(idColumn);
-                columnNumber++;
+            int columnNumber = indexOfIDColumn + 1; // ColumnNumbers start from 1
+
+            // Add the ID-Column, even if the column wasn't included in columnNames
+            ColumnType columnType;
+            if (idColumnFound) {
+                columnType = columnTypes[indexOfIDColumn];
+            } else {
+                columnType = defaultRowKeyColumnType;
             }
+            final MutableColumn idColumn = new MutableColumn(HBaseDataContext.FIELD_ID, columnType)
+                    .setPrimaryKey(true)
+                    .setColumnNumber(columnNumber)
+                    .setTable(this);
+            addColumn(idColumn);
 
+            // Add the other columns
             for (int i = 0; i < columnNames.length; i++) {
                 final String columnName = columnNames[i];
-
+                if (idColumnFound) {
+                    columnNumber = i + 1; // ColumnNumbers start from 1
+                } else {
+                    columnNumber = i + 2; // ColumnNumbers start from 1 + the ID-column has just been created
+                }
                 if (!HBaseDataContext.FIELD_ID.equals(columnName)) {
-                    final ColumnType type = types[i];
+                    final ColumnType type = columnTypes[i];
                     final MutableColumn column = new MutableColumn(columnName, type);
                     column.setTable(this);
                     column.setColumnNumber(columnNumber);
-                    columnNumber++;
                     addColumn(column);
+                    columnNumber++;
                 }
             }
         }
@@ -100,7 +120,9 @@ final class HBaseTable extends MutableTable {
                 int columnNumber = 1;
 
                 final MutableColumn idColumn = new MutableColumn(HBaseDataContext.FIELD_ID, _defaultRowKeyColumnType)
-                        .setPrimaryKey(true).setColumnNumber(columnNumber).setTable(this);
+                        .setPrimaryKey(true)
+                        .setColumnNumber(columnNumber)
+                        .setTable(this);
                 addColumn(idColumn);
                 columnNumber++;
 
@@ -124,4 +146,28 @@ final class HBaseTable extends MutableTable {
         }
         return columnsInternal;
     }
+
+    /**
+     * Check if a list of columnNames all exist in this table
+     * If a column doesn't exist, then a {@link MetaModelException} is thrown
+     * @param columnNamesOfCheckedTable
+     */
+    public void checkForNotMatchingColumns(final List<String> columnNamesOfCheckedTable) {
+        final List<String> columnsNamesOfExistingTable = getColumnNames();
+        for (String columnNameOfCheckedTable : columnNamesOfCheckedTable) {
+            boolean matchingColumnFound = false;
+            int i = 0;
+            while (!matchingColumnFound && i < columnsNamesOfExistingTable.size()) {
+                if (columnNameOfCheckedTable.equals(columnsNamesOfExistingTable.get(i))) {
+                    matchingColumnFound = true;
+                } else {
+                    i++;
+                }
+            }
+            if (!matchingColumnFound) {
+                throw new MetaModelException(String.format("ColumnFamily: %s doesn't exist in the schema of the table",
+                        columnNameOfCheckedTable));
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/3ada7eb4/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
index 5ab38d9..374e325 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
@@ -18,20 +18,39 @@
  */
 package org.apache.metamodel.hbase;
 
+import java.io.IOException;
+
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.drop.AbstractTableDropBuilder;
+import org.apache.metamodel.schema.MutableSchema;
 import org.apache.metamodel.schema.Table;
 
+/**
+ * A builder-class to drop tables in a HBase datastore
+ */
 public class HBaseTableDropBuilder extends AbstractTableDropBuilder {
     private final HBaseUpdateCallback _updateCallback;
 
-    public HBaseTableDropBuilder(Table table, HBaseUpdateCallback updateCallback) {
+    public HBaseTableDropBuilder(final Table table, final HBaseUpdateCallback updateCallback) {
         super(table);
+        if (updateCallback.getDataContext().getDefaultSchema().getTableByName(table.getName()) == null) {
+            throw new MetaModelException("Trying to delete a table that doesn't exist in the datastore.");
+        }
         _updateCallback = updateCallback;
     }
 
     @Override
-    public void execute() throws MetaModelException {
-        _updateCallback.dropTableExecute(getTable());
+    public void execute() {
+        try {
+            // Remove from the datastore
+            final HBaseClient hBaseClient = _updateCallback.getHBaseClient();
+            final Table table = getTable();
+            hBaseClient.dropTable(table.getName());
+
+            // Remove from schema
+            ((MutableSchema) table.getSchema()).removeTable(table);
+        } catch (IOException e) {
+            throw new MetaModelException(e);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/3ada7eb4/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
index 39670db..e548229 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
@@ -18,7 +18,7 @@
  */
 package org.apache.metamodel.hbase;
 
-import java.io.IOException;
+import java.util.Set;
 
 import org.apache.metamodel.AbstractUpdateCallback;
 import org.apache.metamodel.UpdateCallback;
@@ -26,32 +26,35 @@ import org.apache.metamodel.create.TableCreationBuilder;
 import org.apache.metamodel.delete.RowDeletionBuilder;
 import org.apache.metamodel.drop.TableDropBuilder;
 import org.apache.metamodel.insert.RowInsertionBuilder;
-import org.apache.metamodel.schema.MutableSchema;
 import org.apache.metamodel.schema.Schema;
 import org.apache.metamodel.schema.Table;
 
+/**
+ * This class is used to build objects to do client-operations on a HBase datastore
+ */
 public class HBaseUpdateCallback extends AbstractUpdateCallback implements UpdateCallback {
 
-    private final HBaseConfiguration _configuration;
-
-    private final HBaseDataContext _dataContext;
+    private final HBaseClient _hBaseClient;
 
     public HBaseUpdateCallback(HBaseDataContext dataContext) {
         super(dataContext);
-        _configuration = dataContext.getConfiguration();
-        _dataContext = dataContext;
+        _hBaseClient = new HBaseClient(dataContext.getConnection());
     }
 
     @Override
-    public TableCreationBuilder createTable(Schema schema, String name) throws IllegalArgumentException,
-            IllegalStateException {
-        throw new UnsupportedOperationException(
-                "Use createTable(Schema schema, String name, HBaseColumn[] outputColumns)");
+    public TableCreationBuilder createTable(Schema schema, String name) {
+        return new HBaseCreateTableBuilder(this, schema, name);
     }
 
-    public HBaseCreateTableBuilder createTable(Schema schema, String name, HBaseColumn[] outputColumns)
-            throws IllegalArgumentException, IllegalStateException {
-        return new HBaseCreateTableBuilder(this, schema, name, outputColumns);
+    /**
+     * Initiates the building of a table creation operation.
+     * @param schema the schema to create the table in
+     * @param name the name of the new table
+     * @param columnFamilies the columnFamilies of the new table
+     * @return {@link HBaseCreateTableBuilder}
+     */
+    public HBaseCreateTableBuilder createTable(Schema schema, String name, Set<String> columnFamilies) {
+        return new HBaseCreateTableBuilder(this, schema, name, columnFamilies);
     }
 
     @Override
@@ -60,78 +63,34 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
     }
 
     @Override
-    public TableDropBuilder dropTable(Table table) throws IllegalArgumentException, IllegalStateException,
-            UnsupportedOperationException {
+    public TableDropBuilder dropTable(Table table) {
         return new HBaseTableDropBuilder(table, this);
     }
 
-    public void dropTableExecute(Table table) {
-        try {
-            final HBaseWriter HbaseWriter = new HBaseWriter(HBaseDataContext.createConfig(_configuration));
-            HbaseWriter.dropTable(table.getName());
-            MutableSchema schema = (MutableSchema) table.getSchema();
-            schema.removeTable(table);
-        } catch (IOException e) {
-            e.printStackTrace();
-        }
-    }
-
     @Override
-    public RowInsertionBuilder insertInto(Table table) throws IllegalArgumentException, IllegalStateException,
-            UnsupportedOperationException {
-        throw new UnsupportedOperationException("Use insertInto(Table table, HBaseColumn[] outputColumns)");
-    }
-
-    public HBaseRowInsertionBuilder insertInto(Table table, HBaseColumn[] columns) throws IllegalArgumentException {
+    public RowInsertionBuilder insertInto(Table table) {
         if (table instanceof HBaseTable) {
-            return new HBaseRowInsertionBuilder(this, (HBaseTable) table, columns);
-        }
-        throw new IllegalArgumentException("Not an HBase table: " + table);
-    }
-
-    protected synchronized void writeRow(HBaseTable hBaseTable, HBaseColumn[] outputColumns, Object[] values) {
-        try {
-            final HBaseWriter HbaseWriter = new HBaseWriter(HBaseDataContext.createConfig(_configuration));
-            HbaseWriter.writeRow(hBaseTable, outputColumns, values);
-        } catch (IOException e) {
-            e.printStackTrace();
+            return new HBaseRowInsertionBuilder(this, (HBaseTable) table);
+        } else {
+            throw new IllegalArgumentException("Not an HBase table: " + table);
         }
     }
 
     @Override
     public boolean isDeleteSupported() {
-        return false;
+        return true;
     }
 
     @Override
-    public RowDeletionBuilder deleteFrom(Table table) throws IllegalArgumentException, IllegalStateException,
-            UnsupportedOperationException {
-        throw new UnsupportedOperationException(
-                "Use deleteFrom(HBaseUpdateCallback updateCallback, Table table, Object key)");
-    }
-
-    public HBaseRowDeletionBuilder deleteFrom(HBaseUpdateCallback updateCallback, Table table)
-            throws IllegalArgumentException {
+    public RowDeletionBuilder deleteFrom(Table table) {
         if (table instanceof HBaseTable) {
-            return new HBaseRowDeletionBuilder(this, (HBaseTable) table);
+            return new HBaseRowDeletionBuilder(_hBaseClient, (HBaseTable) table);
+        } else {
+            throw new IllegalArgumentException("Not an HBase table: " + table);
         }
-        throw new IllegalArgumentException("Not an HBase table: " + table);
-    }
-
-    protected synchronized void deleteRow(HBaseTable hBaseTable, Object key) {
-        try {
-            final HBaseWriter HbaseWriter = new HBaseWriter(HBaseDataContext.createConfig(_configuration));
-            HbaseWriter.deleteRow(hBaseTable, key);
-        } catch (IOException e) {
-            e.printStackTrace();
-        }
-    }
-
-    public HBaseConfiguration getConfiguration() {
-        return _configuration;
     }
 
-    public HBaseDataContext getDataContext() {
-        return _dataContext;
+    public HBaseClient getHBaseClient() {
+        return _hBaseClient;
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/3ada7eb4/hbase/src/main/java/org/apache/metamodel/hbase/HBaseWriter.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseWriter.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseWriter.java
deleted file mode 100644
index ad63597..0000000
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseWriter.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/**
- * 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.metamodel.hbase;
-
-import java.io.IOException;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metamodel.MetaModelException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public final class HBaseWriter extends Configured {
-
-    private static final Logger logger = LoggerFactory.getLogger(HBaseWriter.class);
-
-    private final Connection _connection;
-
-    public HBaseWriter(Configuration configuration) throws IOException {
-        _connection = ConnectionFactory.createConnection(configuration);
-    }
-
-    public void writeRow(HBaseTable hBaseTable, HBaseColumn[] outputColumns, Object[] values) throws IOException {
-        try {
-            Table table = _connection.getTable(TableName.valueOf(hBaseTable.getName()));
-            try {
-                int indexOfIdColumn = 0;
-                boolean idColumnFound = false;
-                while (!idColumnFound && indexOfIdColumn < outputColumns.length) {
-                    if (outputColumns[indexOfIdColumn].getColumnFamily().equals(HBaseDataContext.FIELD_ID)) {
-                        idColumnFound = true;
-                    } else {
-                        indexOfIdColumn++;
-                    }
-                }
-                if (!idColumnFound) {
-                    throw new MetaModelException("The ID Column family was not found");
-                }
-
-                Put put = new Put(Bytes.toBytes(values[indexOfIdColumn].toString()));
-
-                for (int i = 0; i < outputColumns.length; i++) {
-                    if (!outputColumns[i].getColumnFamily().equals(HBaseDataContext.FIELD_ID)) {
-                        put.addColumn(Bytes.toBytes(outputColumns[i].getColumnFamily()), Bytes.toBytes(outputColumns[i]
-                                .getQualifier()), Bytes.toBytes(values[i].toString()));
-                    }
-                }
-                table.put(put);
-            } finally {
-                table.close();
-            }
-        } finally {
-            _connection.close();
-        }
-    }
-
-    public void deleteRow(HBaseTable hBaseTable, Object key) throws IOException {
-        try {
-            Table table = _connection.getTable(TableName.valueOf(hBaseTable.getName()));
-            try {
-                if (rowExists(table, key) == true) {
-                    table.delete(new Delete(Bytes.toBytes(key.toString())));
-                } else {
-                    logger.warn("Rowkey with value " + key.toString() + " doesn't exist in the table");
-                }
-            } finally {
-                table.close();
-            }
-        } finally {
-            _connection.close();
-        }
-    }
-
-    private boolean rowExists(Table table, Object key) throws IOException {
-        Get get = new Get(Bytes.toBytes(key.toString()));
-        return (table.get(get).isEmpty()) == true ? false : true;
-    }
-
-    public void createTable(String tableName, Set<String> columnFamilies) throws IOException {
-        try {
-            // Create table
-            Admin admin = _connection.getAdmin();
-            try {
-                TableName hBasetableName = TableName.valueOf(tableName);
-                HTableDescriptor tableDescriptor = new HTableDescriptor(hBasetableName);
-                for (String columnFamilie : columnFamilies) {
-                    if (!columnFamilie.equals(HBaseDataContext.FIELD_ID)) {
-                        tableDescriptor.addFamily(new HColumnDescriptor(columnFamilie));
-                    }
-                }
-                admin.createTable(tableDescriptor);
-                HTableDescriptor[] tables = admin.listTables();
-                if (tables.length != 1 && Bytes.equals(hBasetableName.getName(), tables[0].getTableName().getName())) {
-                    throw new IOException("Failed create of table");
-                }
-            } finally {
-                admin.close();
-            }
-        } finally {
-            _connection.close();
-        }
-
-    }
-
-    public void dropTable(String tableName) throws IOException {
-        try {
-            Admin admin = _connection.getAdmin();
-            try {
-                TableName hBasetableName = TableName.valueOf(tableName);
-                admin.disableTable(hBasetableName);
-                admin.deleteTable(hBasetableName);
-            } finally {
-                admin.close();
-            }
-        } finally {
-            _connection.close();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/metamodel/blob/3ada7eb4/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
index fb21b46..3872611 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.metamodel.hbase;
 
+import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -32,43 +33,10 @@ import org.apache.metamodel.util.SimpleTableDef;
 
 public class HBaseDataContextTest extends HBaseTestCase {
 
-    // Table
-    private static final String EXAMPLE_TABLE_NAME = "table_for_junit";
-
-    // ColumnFamilies
-    private static final String CF_FOO = "foo";
-    private static final String CF_BAR = "bar";
-
-    // Qualifiers
-    private static final String Q_HELLO = "hello";
-    private static final String Q_HI = "hi";
-    private static final String Q_HEY = "hey";
-    private static final String Q_BAH = "bah";
-
-    // RowKeys
-    private static final String RK_1 = "junit1";
-    private static final String RK_2 = "junit2";
-
-    private static final int NUMBER_OF_ROWS = 2;
-
-    // Values
-    private static final String V_WORLD = "world";
-    private static final String V_THERE = "there";
-    private static final String V_YO = "yo";
-    private static final byte[] V_123_BYTE_ARRAY = new byte[] { 1, 2, 3 };
-    private static final String V_YOU = "you";
-
-    private HBaseDataContext _dataContext;
-
     @Override
     protected void setUp() throws Exception {
         super.setUp();
         if (isConfigured()) {
-            final String zookeeperHostname = getZookeeperHostname();
-            final int zookeeperPort = getZookeeperPort();
-            final HBaseConfiguration configuration = new HBaseConfiguration(zookeeperHostname, zookeeperPort,
-                    ColumnType.VARCHAR);
-            _dataContext = new HBaseDataContext(configuration);
             createTableNatively();
         }
     }
@@ -80,7 +48,7 @@ public class HBaseDataContextTest extends HBaseTestCase {
         }
 
         // test the schema exploration
-        final Table table = _dataContext.getDefaultSchema().getTableByName(EXAMPLE_TABLE_NAME);
+        final Table table = getDataContext().getDefaultSchema().getTableByName(TABLE_NAME);
         assertNotNull(table);
 
         assertEquals("[" + HBaseDataContext.FIELD_ID + ", " + CF_BAR + ", " + CF_FOO + "]", Arrays.toString(table
@@ -92,7 +60,7 @@ public class HBaseDataContextTest extends HBaseTestCase {
         insertRecordsNatively();
 
         // query using regular configuration
-        final DataSet dataSet1 = _dataContext.query().from(EXAMPLE_TABLE_NAME).selectAll().execute();
+        final DataSet dataSet1 = getDataContext().query().from(TABLE_NAME).selectAll().execute();
         try {
             assertTrue(dataSet1.next());
             assertEquals("Row[values=[" + RK_1 + ", {" + Q_HEY + "=" + V_YO + "," + Q_HI + "=" + V_THERE + "}, {"
@@ -111,14 +79,14 @@ public class HBaseDataContextTest extends HBaseTestCase {
         final String columnName3 = CF_BAR + ":" + Q_HEY;
         final String[] columnNames = new String[] { columnName1, columnName2, columnName3 };
         final ColumnType[] columnTypes = new ColumnType[] { ColumnType.MAP, ColumnType.VARCHAR, ColumnType.VARCHAR };
-        final SimpleTableDef[] tableDefinitions = new SimpleTableDef[] { new SimpleTableDef(EXAMPLE_TABLE_NAME,
+        final SimpleTableDef[] tableDefinitions = new SimpleTableDef[] { new SimpleTableDef(TABLE_NAME,
                 columnNames, columnTypes) };
-        _dataContext = new HBaseDataContext(new HBaseConfiguration("SCH", getZookeeperHostname(), getZookeeperPort(),
-                tableDefinitions, ColumnType.VARCHAR));
+        setDataContext(new HBaseDataContext(new HBaseConfiguration("SCH", getZookeeperHostname(), getZookeeperPort(),
+                tableDefinitions, ColumnType.VARCHAR)));
 
-        final DataSet dataSet2 = _dataContext
+        final DataSet dataSet2 = getDataContext()
                 .query()
-                .from(EXAMPLE_TABLE_NAME)
+                .from(TABLE_NAME)
                 .select(columnName1, columnName2, columnName3)
                 .execute();
         try {
@@ -134,7 +102,7 @@ public class HBaseDataContextTest extends HBaseTestCase {
         }
 
         // query count
-        final DataSet dataSet3 = _dataContext.query().from(EXAMPLE_TABLE_NAME).selectCount().execute();
+        final DataSet dataSet3 = getDataContext().query().from(TABLE_NAME).selectCount().execute();
         try {
             assertTrue(dataSet3.next());
             assertEquals("Row[values=[" + NUMBER_OF_ROWS + "]]", dataSet3.getRow().toString());
@@ -144,9 +112,9 @@ public class HBaseDataContextTest extends HBaseTestCase {
         }
 
         // query only id
-        final DataSet dataSet4 = _dataContext
+        final DataSet dataSet4 = getDataContext()
                 .query()
-                .from(EXAMPLE_TABLE_NAME)
+                .from(TABLE_NAME)
                 .select(HBaseDataContext.FIELD_ID)
                 .execute();
 
@@ -161,9 +129,9 @@ public class HBaseDataContextTest extends HBaseTestCase {
         }
 
         // primary key lookup query - using GET
-        final DataSet dataSet5 = _dataContext
+        final DataSet dataSet5 = getDataContext()
                 .query()
-                .from(EXAMPLE_TABLE_NAME)
+                .from(TABLE_NAME)
                 .select(HBaseDataContext.FIELD_ID)
                 .where(HBaseDataContext.FIELD_ID)
                 .eq(RK_1)
@@ -178,9 +146,8 @@ public class HBaseDataContextTest extends HBaseTestCase {
         }
     }
 
-    private void insertRecordsNatively() throws Exception {
-        final org.apache.hadoop.hbase.client.Table hTable = _dataContext.getHTable(EXAMPLE_TABLE_NAME);
-        try {
+    private void insertRecordsNatively() throws IOException, InterruptedException {
+        try (final org.apache.hadoop.hbase.client.Table hTable = getDataContext().getHTable(TABLE_NAME)) {
             final Put put1 = new Put(RK_1.getBytes());
             put1.addColumn(CF_FOO.getBytes(), Q_HELLO.getBytes(), V_WORLD.getBytes());
             put1.addColumn(CF_BAR.getBytes(), Q_HI.getBytes(), V_THERE.getBytes());
@@ -192,27 +159,26 @@ public class HBaseDataContextTest extends HBaseTestCase {
 
             final Object[] result = new Object[NUMBER_OF_ROWS];
             hTable.batch(Arrays.asList(put1, put2), result);
-        } finally {
-            hTable.close();
         }
     }
 
-    private void createTableNatively() throws Exception {
-        final TableName tableName = TableName.valueOf(EXAMPLE_TABLE_NAME);
-
-        // check if the table exists
-        if (_dataContext.getAdmin().isTableAvailable(tableName)) {
-            System.out.println("Unittest table already exists: " + EXAMPLE_TABLE_NAME);
-            // table already exists
-            return;
+    private void createTableNatively() throws IOException {
+        try (Admin admin = getDataContext().getAdmin()) {
+            final TableName tableName = TableName.valueOf(TABLE_NAME);
+
+            // Check if the table exists
+            if (admin.isTableAvailable(tableName)) {
+                // table already exists
+                System.out.println("Unittest table already exists: " + TABLE_NAME);
+            } else {
+                // Create table
+                System.out.println("Creating table");
+                final HTableDescriptor tableDescriptor = new HTableDescriptor(tableName);
+                tableDescriptor.addFamily(new HColumnDescriptor(CF_FOO.getBytes()));
+                tableDescriptor.addFamily(new HColumnDescriptor(CF_BAR.getBytes()));
+                admin.createTable(tableDescriptor);
+                System.out.println("Created table");
+            }
         }
-
-        Admin admin = _dataContext.getAdmin();
-        System.out.println("Creating table");
-        final HTableDescriptor tableDescriptor = new HTableDescriptor(tableName);
-        tableDescriptor.addFamily(new HColumnDescriptor(CF_FOO.getBytes()));
-        tableDescriptor.addFamily(new HColumnDescriptor(CF_BAR.getBytes()));
-        admin.createTable(tableDescriptor);
-        System.out.println("Created table");
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/3ada7eb4/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
index 1f7f162..b9524a8 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
@@ -22,14 +22,45 @@ import java.io.File;
 import java.io.FileReader;
 import java.util.Properties;
 
+import org.apache.metamodel.schema.ColumnType;
+
 import junit.framework.TestCase;
 
 public abstract class HBaseTestCase extends TestCase {
 
+    // TableName
+    protected static final String TABLE_NAME = "table_for_junit";
+
+    // ColumnFamilies
+    protected static final int NUMBER_OF_CFS = 3; // foo + bar + ID
+    protected static final String CF_FOO = "foo";
+    protected static final String CF_BAR = "bar";
+
+    // Qualifiers
+    protected static final String Q_HELLO = "hello";
+    protected static final String Q_HI = "hi";
+    protected static final String Q_HEY = "hey";
+    protected static final String Q_BAH = "bah";
+
+    // Number of rows
+    protected static final int NUMBER_OF_ROWS = 2;
+
+    // RowKeys
+    protected static final String RK_1 = "junit1";
+    protected static final String RK_2 = "junit2";
+
+    // RowValues
+    protected static final String V_WORLD = "world";
+    protected static final String V_THERE = "there";
+    protected static final String V_YO = "yo";
+    protected static final byte[] V_123_BYTE_ARRAY = new byte[] { 1, 2, 3 };
+    protected static final String V_YOU = "you";
+
     private String zookeeperHostname;
     private int zookeeperPort;
     private boolean _configured;
-    
+    private HBaseDataContext _dataContext;
+
     @Override
     protected void setUp() throws Exception {
         super.setUp();
@@ -43,13 +74,18 @@ public abstract class HBaseTestCase extends TestCase {
             if (zookeeperPortPropertyValue != null && !zookeeperPortPropertyValue.isEmpty()) {
                 zookeeperPort = Integer.parseInt(zookeeperPortPropertyValue);
             }
-            
+
             _configured = (zookeeperHostname != null && !zookeeperHostname.isEmpty());
         } else {
             _configured = false;
         }
+        if (isConfigured()) {
+            final HBaseConfiguration configuration = new HBaseConfiguration(zookeeperHostname, zookeeperPort,
+                    ColumnType.VARCHAR);
+            setDataContext(new HBaseDataContext(configuration));
+        }
     }
-    
+
     private String getPropertyFilePath() {
         String userHome = System.getProperty("user.home");
         return userHome + "/metamodel-integrationtest-configuration.properties";
@@ -63,7 +99,7 @@ public abstract class HBaseTestCase extends TestCase {
     public boolean isConfigured() {
         return _configured;
     }
-    
+
     public String getZookeeperHostname() {
         return zookeeperHostname;
     }
@@ -71,4 +107,18 @@ public abstract class HBaseTestCase extends TestCase {
     public int getZookeeperPort() {
         return zookeeperPort;
     }
+
+    public HBaseDataContext getDataContext() {
+        return _dataContext;
+    }
+
+    public void setDataContext(HBaseDataContext dataContext) {
+        this._dataContext = dataContext;
+    }
+
+    @Override
+    protected void tearDown() throws Exception {
+        super.tearDown();
+        _dataContext.getConnection().close();
+    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/3ada7eb4/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
new file mode 100644
index 0000000..359be01
--- /dev/null
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
@@ -0,0 +1,213 @@
+/**
+ * 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.metamodel.hbase;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.metamodel.MetaModelException;
+import org.apache.metamodel.schema.ColumnType;
+import org.apache.metamodel.schema.ImmutableSchema;
+import org.apache.metamodel.schema.MutableSchema;
+import org.apache.metamodel.schema.Table;
+import org.apache.metamodel.util.SimpleTableDef;
+
+public class HBaseUpdateCallbackTest extends HBaseTestCase {
+
+    private HBaseUpdateCallback updateCallback;
+    private MutableSchema schema;
+
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();
+        if (isConfigured()) {
+            updateCallback = new HBaseUpdateCallback(getDataContext());
+            schema = (MutableSchema) getDataContext().getDefaultSchema();
+
+            if (schema.getTableByName(TABLE_NAME) != null) {
+                dropTableIfItExists();
+            }
+        }
+    }
+
+    public void testDropTable() throws IOException {
+        dropTableIfItExists();
+
+        try {
+            HBaseTable table = createHBaseTable();
+            updateCallback.dropTable(table).execute();
+            fail("Should get an exception that the table doesn't exist in the datastore");
+        } catch (MetaModelException e) {
+            assertEquals("Trying to delete a table that doesn't exist in the datastore.", e.getMessage());
+        }
+    }
+
+    private void dropTableIfItExists() {
+        Table table = schema.getTableByName(TABLE_NAME);
+        if (table != null) {
+            updateCallback.dropTable(table).execute();
+            // Check schema
+            assertNull(schema.getTableByName(TABLE_NAME));
+            // Check in the datastore
+            try (Admin admin = getDataContext().getAdmin()) {
+                assertFalse(admin.tableExists(TableName.valueOf(TABLE_NAME)));
+            } catch (IOException e) {
+                fail("Should not an exception checking if the table exists");
+            }
+        }
+    }
+
+    public void testCreateTable() {
+        // Drop the table if it exists
+        dropTableIfItExists();
+
+        // Test 1: Create a table with an immutableSchema, should throw a IllegalArgumentException
+        ImmutableSchema immutableSchema = new ImmutableSchema(schema);
+        try {
+            updateCallback.createTable(immutableSchema, TABLE_NAME).execute();
+            fail("Should get an exception that the schema isn't mutable");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Not a mutable schema: " + immutableSchema, e.getMessage());
+        }
+
+        // Test 2: Create a table without columnFamilies, should throw a MetaModelException
+        try {
+            updateCallback.createTable(schema, TABLE_NAME).execute();
+            fail("Should get an exception that the columnFamilies haven't been set");
+        } catch (MetaModelException e) {
+            assertEquals("Creating a table without columnFamilies", e.getMessage());
+        }
+
+        // Test 3: Create a table with columnFamilies null, should throw a MetaModelException
+        try {
+            updateCallback.createTable(schema, TABLE_NAME, null).execute();
+            fail("Should get an exception that the columnFamilies haven't been set");
+        } catch (MetaModelException e) {
+            assertEquals("Creating a table without columnFamilies", e.getMessage());
+        }
+
+        // Test 4: Create a table with columnFamilies empty, should throw a MetaModelException
+        try {
+            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<String>();
+            updateCallback.createTable(schema, TABLE_NAME, columnFamilies).execute();
+            fail("Should get an exception that the columnFamilies haven't been set");
+        } catch (MetaModelException e) {
+            assertEquals("Creating a table without columnFamilies", e.getMessage());
+        }
+
+        HBaseTable table = createHBaseTable();
+
+        // Test 5: Create a table without the ID-Column, should throw a MetaModelException
+        ArrayList<HBaseColumn> hBaseColumnsAsArrayList = createListWithHBaseColumnsExcludingIDColumn(table);
+        HBaseColumn[] hBaseColumnsAsArray = convertToHBaseColumnArray(hBaseColumnsAsArrayList);
+        Set<String> columnFamilies = HBaseColumn.getColumnFamilies(hBaseColumnsAsArray);
+        try {
+            HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) updateCallback.createTable(
+                    schema, TABLE_NAME);
+
+            hBaseCreateTableBuilder.setColumnFamilies(columnFamilies);
+            hBaseCreateTableBuilder.execute();
+            fail("Should get an exception that the ID-colum is missing");
+        } catch (MetaModelException e) {
+            assertEquals("ColumnFamily: " + HBaseDataContext.FIELD_ID + " not found", e.getMessage());
+        }
+
+        // Test 6: Create a table including the ID-Column (columnFamilies not in constructor), should work
+        hBaseColumnsAsArrayList = createListWithHBaseColumnsIncludingIDColumn(table);
+        hBaseColumnsAsArray = convertToHBaseColumnArray(hBaseColumnsAsArrayList);
+        columnFamilies = HBaseColumn.getColumnFamilies(hBaseColumnsAsArray);
+        try {
+            HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) updateCallback.createTable(
+                    schema, TABLE_NAME);
+
+            hBaseCreateTableBuilder.setColumnFamilies(HBaseColumn.getColumnFamilies(hBaseColumnsAsArray));
+            hBaseCreateTableBuilder.execute();
+            checkSuccesfullyInsertedTable();
+        } catch (Exception e) {
+            fail("Should not get an exception");
+        }
+        dropTableIfItExists();
+
+        // Test 7: Create a table including the ID-Column (columnFamilies in constructor), should work
+        try {
+            updateCallback.createTable(schema, TABLE_NAME, columnFamilies).execute();
+            checkSuccesfullyInsertedTable();
+        } catch (Exception e) {
+            fail("Should not get an exception");
+        }
+        dropTableIfItExists();
+    }
+
+    private void checkSuccesfullyInsertedTable() throws IOException {
+        // Check the schema
+        assertNotNull(schema.getTableByName(TABLE_NAME));
+        // Check in the datastore
+        try (Admin admin = getDataContext().getAdmin()) {
+            assertTrue(admin.tableExists(TableName.valueOf(TABLE_NAME)));
+        } catch (IOException e) {
+            fail("Should not an exception checking if the table exists");
+        }
+    }
+
+    // public void testInsertRows() throws IOException {
+    // // Drop the table if it exists
+    // dropTableIfItExists();
+    //
+    // insertTable();
+    // }
+
+    private void insertTable() throws IOException {
+        HBaseTable table = createHBaseTable();
+        ArrayList<HBaseColumn> hBaseColumnsAsArrayList = createListWithHBaseColumnsIncludingIDColumn(table);
+        HBaseColumn[] hBaseColumnsAsArray = convertToHBaseColumnArray(hBaseColumnsAsArrayList);
+        Set<String> columnFamilies = HBaseColumn.getColumnFamilies(hBaseColumnsAsArray);
+        updateCallback.createTable(schema, TABLE_NAME, columnFamilies).execute();
+        checkSuccesfullyInsertedTable();
+    }
+
+    private HBaseTable createHBaseTable() {
+        String[] columnNames = new String[] { CF_FOO, CF_BAR };
+        ColumnType[] columnTypes = new ColumnType[] { ColumnType.STRING, ColumnType.STRING };
+        SimpleTableDef tableDef = new SimpleTableDef(TABLE_NAME, columnNames, columnTypes);
+        return new HBaseTable(getDataContext(), tableDef, schema, ColumnType.STRING);
+    }
+
+    private static ArrayList<HBaseColumn> createListWithHBaseColumnsExcludingIDColumn(final HBaseTable table) {
+        ArrayList<HBaseColumn> hbaseColumns = new ArrayList<HBaseColumn>();
+        hbaseColumns.add(new HBaseColumn(CF_FOO, Q_HELLO, table));
+        hbaseColumns.add(new HBaseColumn(CF_FOO, Q_HI, table));
+        hbaseColumns.add(new HBaseColumn(CF_BAR, Q_HEY, table));
+        hbaseColumns.add(new HBaseColumn(CF_BAR, Q_BAH, table));
+        return hbaseColumns;
+    }
+
+    private static ArrayList<HBaseColumn> createListWithHBaseColumnsIncludingIDColumn(final HBaseTable table) {
+        ArrayList<HBaseColumn> hbaseColumns = createListWithHBaseColumnsExcludingIDColumn(table);
+        hbaseColumns.add(new HBaseColumn(HBaseDataContext.FIELD_ID, table));
+        return hbaseColumns;
+    }
+
+    private static HBaseColumn[] convertToHBaseColumnArray(final ArrayList<HBaseColumn> hBaseColumnsAsArrayList) {
+        return hBaseColumnsAsArrayList.toArray(new HBaseColumn[hBaseColumnsAsArrayList.size()]);
+    }
+}


[19/31] metamodel git commit: HBase improvements - Refactoring (mostly small improvements, the getHBaseClient method has been moved to HBaseDataContent)

Posted by ka...@apache.org.
HBase improvements - Refactoring (mostly small improvements, the getHBaseClient method has been moved to HBaseDataContent)


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/4a928d17
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/4a928d17
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/4a928d17

Branch: refs/heads/master
Commit: 4a928d175897f6d71738a7113663723d160ffa0b
Parents: f48e9c8
Author: Gerard Dellemann <g....@quadient.com>
Authored: Mon Jun 4 01:20:09 2018 +0200
Committer: Gerard Dellemann <g....@quadient.com>
Committed: Mon Jun 4 01:20:09 2018 +0200

----------------------------------------------------------------------
 .../org/apache/metamodel/hbase/HBaseClient.java |  58 ++++++---
 .../org/apache/metamodel/hbase/HBaseColumn.java |   5 +-
 .../hbase/HBaseCreateTableBuilder.java          |   9 +-
 .../metamodel/hbase/HBaseDataContext.java       |  14 +--
 .../hbase/HBaseRowDeletionBuilder.java          |  14 +--
 .../hbase/HBaseRowInsertionBuilder.java         |  29 ++---
 .../org/apache/metamodel/hbase/HBaseTable.java  |   7 +-
 .../metamodel/hbase/HBaseTableDropBuilder.java  |   2 +-
 .../metamodel/hbase/HBaseUpdateCallback.java    |  11 +-
 .../apache/metamodel/hbase/CreateTableTest.java |   8 +-
 .../apache/metamodel/hbase/DeleteRowTest.java   |  16 +--
 .../hbase/HBaseUpdateCallbackTest.java          |  44 +++++--
 .../apache/metamodel/hbase/InsertRowTest.java   | 118 ++++++++++---------
 13 files changed, 193 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/4a928d17/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
index 8eda465..a9040fc 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
@@ -38,13 +38,13 @@ import org.slf4j.LoggerFactory;
 /**
  * This class can perform client-operations on a HBase datastore
  */
-public final class HBaseClient {
+final class HBaseClient {
 
     private static final Logger logger = LoggerFactory.getLogger(HBaseClient.class);
 
     private final Connection _connection;
 
-    public HBaseClient(Connection connection) {
+    public HBaseClient(final Connection connection) {
         this._connection = connection;
     }
 
@@ -55,24 +55,37 @@ public final class HBaseClient {
      * @param values
      * @throws IllegalArgumentException when any parameter is null or the indexOfIdColumn is impossible
      * @throws MetaModelException when no ID-column is found.
-     * @throws MetaModelException when a {@link IOException} is catched
+     * @throws MetaModelException when a {@link IOException} is caught
      */
-    // TODO: Use the ColumnTypes to determine the inserts. Now the toString() method is called on the object.
-    public void insertRow(String tableName, HBaseColumn[] columns, Object[] values, int indexOfIdColumn) {
+    public void insertRow(final String tableName, final HBaseColumn[] columns, final Object[] values,
+            final int indexOfIdColumn) {
         if (tableName == null || columns == null || values == null || indexOfIdColumn >= values.length
                 || values[indexOfIdColumn] == null) {
             throw new IllegalArgumentException(
                     "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
         }
+        if (columns.length != values.length) {
+            throw new IllegalArgumentException("The amount of columns don't match the amount of values");
+        }
         try (final Table table = _connection.getTable(TableName.valueOf(tableName))) {
             // Create a put with the values of indexOfIdColumn as rowkey
-            final Put put = new Put(Bytes.toBytes(values[indexOfIdColumn].toString()));
+            final Put put = new Put(getValueAsByteArray(values[indexOfIdColumn]));
 
             // Add the other values to the put
             for (int i = 0; i < columns.length; i++) {
                 if (i != indexOfIdColumn) {
-                    put.addColumn(Bytes.toBytes(columns[i].getColumnFamily()), Bytes.toBytes(columns[i].getQualifier()),
-                            Bytes.toBytes(values[i].toString()));
+                    // NullChecker is already forced within the HBaseColumn class
+                    final byte[] columnFamily = Bytes.toBytes(columns[i].getColumnFamily());
+                    // An HBaseColumn doesn't need a qualifier, this only works when the qualifier is empty (not null).
+                    // Otherwise NullPointer exceptions will happen
+                    byte[] qualifier = null;
+                    if (columns[i].getQualifier() != null) {
+                        qualifier = Bytes.toBytes(columns[i].getQualifier());
+                    } else {
+                        qualifier = Bytes.toBytes(new String(""));
+                    }
+                    final byte[] value = getValueAsByteArray(values[i]);
+                    put.addColumn(columnFamily, qualifier, value);
                 }
             }
             // Add the put to the table
@@ -87,13 +100,13 @@ public final class HBaseClient {
      * @param tableName
      * @param rowKey
      * @throws IllegalArgumentException when any parameter is null
-     * @throws MetaModelException when a {@link IOException} is catched
+     * @throws MetaModelException when a {@link IOException} is caught
      */
-    public void deleteRow(String tableName, Object rowKey) {
+    public void deleteRow(final String tableName, final Object rowKey) {
         if (tableName == null || rowKey == null) {
             throw new IllegalArgumentException("Can't delete a row without having tableName or rowKey");
         }
-        byte[] rowKeyAsByteArray = Bytes.toBytes(rowKey.toString());
+        byte[] rowKeyAsByteArray = getValueAsByteArray(rowKey);
         if (rowKeyAsByteArray.length > 0) {
             try (final Table table = _connection.getTable(TableName.valueOf(tableName));) {
                 if (rowExists(table, rowKeyAsByteArray) == true) {
@@ -116,7 +129,7 @@ public final class HBaseClient {
      * @return boolean
      * @throws IOException
      */
-    private boolean rowExists(Table table, byte[] rowKey) throws IOException {
+    private boolean rowExists(final Table table, final byte[] rowKey) throws IOException {
         final Get get = new Get(rowKey);
         return !table.get(get).isEmpty();
     }
@@ -126,7 +139,7 @@ public final class HBaseClient {
      * @param tableName
      * @param columnFamilies
      * @throws IllegalArgumentException when any parameter is null
-     * @throws MetaModelException when a {@link IOException} is catched
+     * @throws MetaModelException when a {@link IOException} is caught
      */
     public void createTable(String tableName, Set<String> columnFamilies) {
         if (tableName == null || columnFamilies == null || columnFamilies.isEmpty()) {
@@ -152,9 +165,9 @@ public final class HBaseClient {
      * Disable and drop a table from a HBase datastore
      * @param tableName
      * @throws IllegalArgumentException when tableName is null
-     * @throws MetaModelException when a {@link IOException} is catched
+     * @throws MetaModelException when a {@link IOException} is caught
      */
-    public void dropTable(String tableName) {
+    public void dropTable(final String tableName) {
         if (tableName == null) {
             throw new IllegalArgumentException("Can't drop a table without having the tableName");
         }
@@ -166,4 +179,19 @@ public final class HBaseClient {
             throw new MetaModelException(e);
         }
     }
+
+    /**
+     * Converts a Object value into a byte array, if it isn't a byte array already
+     * @param value
+     * @return value as a byte array
+     */
+    private byte[] getValueAsByteArray(final Object value) {
+        byte[] valueAsByteArray;
+        if (value instanceof byte[]) {
+            valueAsByteArray = (byte[]) value;
+        } else {
+            valueAsByteArray = Bytes.toBytes(value.toString());
+        }
+        return valueAsByteArray;
+    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/4a928d17/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
index 4470e10..45e7f4f 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
@@ -25,9 +25,9 @@ import org.apache.metamodel.schema.SuperColumnType;
 import org.apache.metamodel.schema.Table;
 
 public final class HBaseColumn extends MutableColumn {
-    public final static ColumnType DEFAULT_COLUMN_TYPE_FOR_ID_COLUMN = new ColumnTypeImpl("BYTE[]",
+    public static final ColumnType DEFAULT_COLUMN_TYPE_FOR_ID_COLUMN = new ColumnTypeImpl("BYTE[]",
             SuperColumnType.LITERAL_TYPE);
-    public final static ColumnType DEFAULT_COLUMN_TYPE_FOR_COLUMN_FAMILIES = ColumnType.LIST;
+    public static final ColumnType DEFAULT_COLUMN_TYPE_FOR_COLUMN_FAMILIES = ColumnType.LIST;
 
     private final String columnFamily;
     private final String qualifier;
@@ -59,6 +59,7 @@ public final class HBaseColumn extends MutableColumn {
 
         this.columnFamily = columnFamily;
         this.qualifier = qualifier;
+
         setColumnNumber(columnNumber);
         setPrimaryKey(HBaseDataContext.FIELD_ID.equals(columnFamily));
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/4a928d17/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
index db4463b..a7c760b 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
@@ -57,14 +57,15 @@ public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseU
 
     @Override
     public Table execute() {
-        if (_columnFamilies == null || _columnFamilies.size() == 0) {
+        if (_columnFamilies == null || _columnFamilies.isEmpty()) {
             throw new MetaModelException("Creating a table without columnFamilies");
         }
 
         final Table table = getTable();
 
         // Add the table to the datastore
-        getUpdateCallback().getHBaseClient().createTable(table.getName(), _columnFamilies);
+        ((HBaseDataContext) getUpdateCallback().getDataContext()).getHBaseClient().createTable(table.getName(),
+                _columnFamilies);
 
         // Update the schema
         addNewTableToSchema(table);
@@ -72,7 +73,9 @@ public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseU
     }
 
     /**
-     * Set the columnFamilies
+     * Set the columnFamilies. This should be used when creating this object using the 
+     * {@link HBaseCreateTableBuilder#HBaseCreateTableBuilder(HBaseUpdateCallback, Schema, String)} 
+     * constructor
      * @param columnFamilies
      */
     public void setColumnFamilies(Set<String> columnFamilies) {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/4a928d17/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
index b913ca0..0386ec3 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
@@ -50,16 +50,12 @@ import org.apache.metamodel.schema.Schema;
 import org.apache.metamodel.schema.Table;
 import org.apache.metamodel.util.FileHelper;
 import org.apache.metamodel.util.SimpleTableDef;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * MetaModel adaptor for Apache HBase.
  */
 public class HBaseDataContext extends QueryPostprocessDataContext implements UpdateableDataContext {
 
-    private static final Logger logger = LoggerFactory.getLogger(HBaseDataContext.class);
-
     public static final String FIELD_ID = "_id";
 
     private final HBaseConfiguration _configuration;
@@ -97,7 +93,7 @@ public class HBaseDataContext extends QueryPostprocessDataContext implements Upd
         }
     }
 
-    static protected Configuration createConfig(HBaseConfiguration configuration) {
+    protected static Configuration createConfig(HBaseConfiguration configuration) {
         Configuration config = org.apache.hadoop.hbase.HBaseConfiguration.create();
         config.set("hbase.zookeeper.quorum", configuration.getZookeeperHostname());
         config.set("hbase.zookeeper.property.clientPort", Integer.toString(configuration.getZookeeperPort()));
@@ -125,7 +121,7 @@ public class HBaseDataContext extends QueryPostprocessDataContext implements Upd
     }
 
     @Override
-    public Schema getMainSchema() throws MetaModelException {
+    protected Schema getMainSchema() throws MetaModelException {
         final MutableSchema schema = new MutableSchema(_configuration.getSchemaName());
 
         SimpleTableDef[] tableDefinitions = _configuration.getTableDefinitions();
@@ -137,7 +133,7 @@ public class HBaseDataContext extends QueryPostprocessDataContext implements Upd
                     SimpleTableDef emptyTableDef = new SimpleTableDef(tables[i].getNameAsString(), new String[0]);
                     tableDefinitions[i] = emptyTableDef;
                 }
-            } catch (IllegalArgumentException | IOException e) {
+            } catch (IOException e) {
                 throw new MetaModelException(e);
             }
         }
@@ -254,4 +250,8 @@ public class HBaseDataContext extends QueryPostprocessDataContext implements Upd
 
         return callback.getUpdateSummary();
     }
+
+    public HBaseClient getHBaseClient() {
+        return new HBaseClient(this.getConnection());
+    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/4a928d17/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
index 99d2571..9b7f9ab 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
@@ -27,7 +27,7 @@ import org.apache.metamodel.schema.Table;
  */
 public class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
 
-    private HBaseClient _hBaseClient;
+    private HBaseDataContext _dataContext;
     private Object _key;
 
     /**
@@ -36,12 +36,12 @@ public class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
      * @param table
      * @throws IllegalArgumentException when the hBaseWriter is null
      */
-    public HBaseRowDeletionBuilder(final HBaseClient hBaseWriter, final Table table) {
+    public HBaseRowDeletionBuilder(final HBaseDataContext dataContext, final Table table) {
         super(table);
-        if (hBaseWriter == null) {
+        if (dataContext == null) {
             throw new IllegalArgumentException("hBaseClient cannot be null");
         }
-        this._hBaseClient = hBaseWriter;
+        this._dataContext = dataContext;
     }
 
     /** 
@@ -52,10 +52,10 @@ public class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
         if (_key == null) {
             throw new MetaModelException("Key cannot be null");
         }
-        _hBaseClient.deleteRow(getTable().getName(), _key);
+        _dataContext.getHBaseClient().deleteRow(getTable().getName(), _key);
     }
 
-    public void setKey(Object _key) {
-        this._key = _key;
+    public void setKey(Object key) {
+        this._key = key;
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/4a928d17/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
index 4ae97c1..b6f797a 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
@@ -35,7 +35,7 @@ import org.apache.metamodel.schema.Column;
 // TODO: Possible future improvement: Make it possible to change the columns for each execute.
 // Now each row will get exactly the same columns.
 public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
-    final int _indexOfIdColumn;
+    private final int _indexOfIdColumn;
 
     /**
      * Creates a {@link HBaseRowInsertionBuilder}. The table and the column's columnFamilies are checked to exist in the schema.
@@ -48,10 +48,6 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
     public HBaseRowInsertionBuilder(final HBaseUpdateCallback updateCallback, final HBaseTable table,
             final List<HBaseColumn> columns) {
         super(updateCallback, table, columns.stream().map(column -> (Column) column).collect(Collectors.toList()));
-        if (columns.isEmpty()) { // TODO: Columns null will already result in a NullPointer at the super. Should the
-                                 // super get a extra check?
-            throw new IllegalArgumentException("The hbaseColumns list is null or empty");
-        }
 
         this._indexOfIdColumn = getIndexOfIdColumn(columns);
         if (_indexOfIdColumn == -1) {
@@ -59,7 +55,8 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
         }
 
         checkTable(updateCallback, table);
-        checkForNotMatchingColumnFamilies(table, getColumnFamilies(columns));
+        // The columns parameter should match the table's columns, just to be sure, this is checked again
+        checkColumnFamilies(table, getColumnFamilies(columns));
     }
 
     /**
@@ -91,23 +88,23 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
             throw new MetaModelException("Trying to insert data into table: " + tableGettingInserts.getName()
                     + ", which doesn't exist yet");
         }
-        checkForNotMatchingColumnFamilies(tableInSchema, tableGettingInserts.getColumnFamilies());
+        checkColumnFamilies(tableInSchema, tableGettingInserts.getColumnFamilies());
     }
 
     /**
      * Check if a list of columnNames all exist in this table
-     *
-     * @param columnNamesOfCheckedTable
+     * @param table Checked tabled
+     * @param columnFamilyNamesOfCheckedTable
      * @throws MetaModelException If a column doesn't exist
      */
-    public void checkForNotMatchingColumnFamilies(final HBaseTable table, final Set<String> columnNamesOfCheckedTable) {
+    public void checkColumnFamilies(final HBaseTable table, final Set<String> columnFamilyNamesOfCheckedTable) {
         Set<String> columnFamilyNamesOfExistingTable = table.getColumnFamilies();
 
-        for (String columnNameOfCheckedTable : columnNamesOfCheckedTable) {
+        for (String columnNameOfCheckedTable : columnFamilyNamesOfCheckedTable) {
             boolean matchingColumnFound = false;
-            Iterator<String> iterator = columnFamilyNamesOfExistingTable.iterator();
-            while (!matchingColumnFound && iterator.hasNext()) {
-                if (columnNameOfCheckedTable.equals(iterator.next())) {
+            Iterator<String> columnFamilies = columnFamilyNamesOfExistingTable.iterator();
+            while (!matchingColumnFound && columnFamilies.hasNext()) {
+                if (columnNameOfCheckedTable.equals(columnFamilies.next())) {
                     matchingColumnFound = true;
                 }
             }
@@ -130,8 +127,8 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
 
     @Override
     public synchronized void execute() {
-        getUpdateCallback().getHBaseClient().insertRow(getTable().getName(), getColumns(), getValues(),
-                _indexOfIdColumn);
+        ((HBaseDataContext) getUpdateCallback().getDataContext()).getHBaseClient().insertRow(getTable().getName(),
+                getColumns(), getValues(), _indexOfIdColumn);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/metamodel/blob/4a928d17/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
index c080856..dab24b0 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
@@ -58,15 +58,14 @@ final class HBaseTable extends MutableTable {
         super(tableDef.getName(), TableType.TABLE, schema);
         _dataContext = dataContext;
         _defaultRowKeyColumnType = defaultRowKeyColumnType;
-        addColumns(tableDef, defaultRowKeyColumnType);
+        addColumns(tableDef);
     }
 
     /**
      * Add multiple columns to this table
      * @param tableDef
-     * @param defaultRowKeyColumnType
      */
-    private void addColumns(final SimpleTableDef tableDef, final ColumnType defaultRowKeyColumnType) {
+    private void addColumns(final SimpleTableDef tableDef) {
         // Add the columns
         final String[] columnNames = tableDef.getColumnNames();
         if (columnNames == null || columnNames.length == 0) {
@@ -82,7 +81,7 @@ final class HBaseTable extends MutableTable {
             if (idColumnFound) {
                 addColumn(HBaseDataContext.FIELD_ID, columnTypes[indexOfIDColumn], indexOfIDColumn + 1);
             } else {
-                addColumn(HBaseDataContext.FIELD_ID, defaultRowKeyColumnType, 1);
+                addColumn(HBaseDataContext.FIELD_ID, _defaultRowKeyColumnType, 1);
             }
 
             // Add the other columns

http://git-wip-us.apache.org/repos/asf/metamodel/blob/4a928d17/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
index a08ab83..0e1d80c 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
@@ -41,7 +41,7 @@ public class HBaseTableDropBuilder extends AbstractTableDropBuilder {
     public void execute() {
         // Remove from the datastore
         final Table table = getTable();
-        _updateCallback.getHBaseClient().dropTable(table.getName());
+        ((HBaseDataContext) _updateCallback.getDataContext()).getHBaseClient().dropTable(table.getName());
 
         // Remove from schema
         ((MutableSchema) table.getSchema()).removeTable(table);

http://git-wip-us.apache.org/repos/asf/metamodel/blob/4a928d17/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
index 9273b07..9c17ddd 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
@@ -35,11 +35,8 @@ import org.apache.metamodel.schema.Table;
  */
 public class HBaseUpdateCallback extends AbstractUpdateCallback implements UpdateCallback {
 
-    private final HBaseClient _hBaseClient;
-
     public HBaseUpdateCallback(final HBaseDataContext dataContext) {
         super(dataContext);
-        _hBaseClient = new HBaseClient(dataContext.getConnection());
     }
 
     @Override
@@ -70,7 +67,7 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
     }
 
     /**
-     * @throws IllegalArgumentException when table isn't a {@link HBaseTable}
+     * @throws UnsupportedOperationException use {@link HBaseUpdateCallback#insertInto(String, String)}
      */
     @Override
     public RowInsertionBuilder insertInto(final Table table) {
@@ -107,13 +104,9 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
     @Override
     public RowDeletionBuilder deleteFrom(final Table table) {
         if (table instanceof HBaseTable) {
-            return new HBaseRowDeletionBuilder(_hBaseClient, table);
+            return new HBaseRowDeletionBuilder(((HBaseDataContext) getDataContext()), table);
         } else {
             throw new IllegalArgumentException("Not an HBase table: " + table);
         }
     }
-
-    public HBaseClient getHBaseClient() {
-        return _hBaseClient;
-    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/4a928d17/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
index b619403..289212b 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
@@ -145,7 +145,7 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
     @Test
     public void testCreateTableWithoutIDColumn() throws IOException {
         final HBaseTable table = createHBaseTable(TABLE_NAME, null, CF_FOO, CF_BAR, null);
-        final LinkedHashMap<HBaseColumn, Object> row = createRow(table, null, CF_FOO, CF_BAR);
+        final LinkedHashMap<HBaseColumn, Object> row = createRow(table, null, CF_FOO, CF_BAR, false);
         final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
         try {
             final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
@@ -165,7 +165,8 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
     @Test
     public void testSettingColumnFamiliesAfterConstrutor() {
         final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-        final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+        final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
+                false);
         final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
         try {
             final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
@@ -185,7 +186,8 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
     @Test
     public void testCreateTableColumnFamiliesInConstrutor() {
         final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-        final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+        final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
+                false);
         final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
         try {
             getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();

http://git-wip-us.apache.org/repos/asf/metamodel/blob/4a928d17/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
index 3cf1ee4..7b7eeab 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
@@ -121,12 +121,12 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
             final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                     CF_BAR);
 
-            checkRows(false);
+            checkRows(false, false);
             final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
                     existingTable);
             rowDeletionBuilder.setKey(RK_1);
             rowDeletionBuilder.execute();
-            checkRows(false);
+            checkRows(false, false);
         } catch (Exception e) {
             fail("Should not get an exception that the row doesn't exist.");
         }
@@ -141,12 +141,12 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
             final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                     CF_BAR);
 
-            checkRows(false);
+            checkRows(false, false);
             final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
                     existingTable);
             rowDeletionBuilder.setKey("");
             rowDeletionBuilder.execute();
-            checkRows(false);
+            checkRows(false, false);
         } catch (Exception e) {
             fail("Should not get an exception that the rowkey is empty.");
         }
@@ -161,19 +161,19 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
             final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                     CF_BAR);
             final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
+                    CF_BAR, false);
             final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
 
-            checkRows(false);
+            checkRows(false, false);
             final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
             setValuesInInsertionBuilder(row, rowInsertionBuilder);
             rowInsertionBuilder.execute();
-            checkRows(true);
+            checkRows(true, false);
             final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
                     existingTable);
             rowDeletionBuilder.setKey(RK_1);
             rowDeletionBuilder.execute();
-            checkRows(false);
+            checkRows(false, false);
         } catch (Exception e) {
             fail("Should not get an exception on deleting a row.");
         }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/4a928d17/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
index b1e503f..425100f 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.*;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
@@ -156,10 +157,11 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
      * @param idColumn ID-column, can be set to null to create a row without this column
      * @param columnFamily1 required columnFamily 1
      * @param columnFamily2 required columnFamily 1
+     * @param qualifiersNull true will create all {@link HBaseColumn}'s with qualifier null
      * @return {@link LinkedHashMap}<{@link HBaseColumn}, {@link Object}>
      */
     protected static LinkedHashMap<HBaseColumn, Object> createRow(final HBaseTable table, final String idColumn,
-            final String columnFamily1, final String columnFamily2) {
+            final String columnFamily1, final String columnFamily2, final boolean qualifiersNull) {
         final LinkedHashMap<HBaseColumn, Object> map = new LinkedHashMap<>();
 
         // Columns
@@ -167,10 +169,17 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         if (idColumn != null) {
             columns.add(new HBaseColumn(idColumn, table));
         }
-        columns.add(new HBaseColumn(columnFamily1, Q_HELLO, table));
-        columns.add(new HBaseColumn(columnFamily1, Q_HI, table));
-        columns.add(new HBaseColumn(columnFamily2, Q_HEY, table));
-        columns.add(new HBaseColumn(columnFamily2, Q_BAH, table));
+        if (!qualifiersNull) {
+            columns.add(new HBaseColumn(columnFamily1, Q_HELLO, table));
+            columns.add(new HBaseColumn(columnFamily1, Q_HI, table));
+            columns.add(new HBaseColumn(columnFamily2, Q_HEY, table));
+            columns.add(new HBaseColumn(columnFamily2, Q_BAH, table));
+        } else {
+            columns.add(new HBaseColumn(columnFamily1, null, table));
+            columns.add(new HBaseColumn(columnFamily1, null, table));
+            columns.add(new HBaseColumn(columnFamily2, null, table));
+            columns.add(new HBaseColumn(columnFamily2, null, table));
+        }
 
         // Values
         final ArrayList<Object> values = new ArrayList<>();
@@ -205,6 +214,7 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
      * Set the values of a {@link HBaseRowInsertionBuilder} from the values in the mapped row
      * @param row {@link LinkedHashMap}<{@link HBaseColumn}, {@link Object}> containing the values
      * @param rowInsertionBuilder insertionBuilder to be set
+     * @param enoughMatchingValues if true, the amount of columns match the amount of values
      */
     protected void setValuesInInsertionBuilder(final LinkedHashMap<HBaseColumn, Object> row,
             final HBaseRowInsertionBuilder rowInsertionBuilder) {
@@ -215,23 +225,35 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         }
     }
 
+    protected Collection<Object> getToLittleValues(final LinkedHashMap<HBaseColumn, Object> row) {
+        Collection<Object> values = row.values();
+        values.remove(V_123_BYTE_ARRAY);
+        return values;
+    }
+
     /**
      * Checks that the table does or doesn't have rows, depending on the rowsExists parameter
      * @param rowsExist true, check that the rows exists. false, check that the result is empty.
+     * @param qualifierNull true, check the results when the qualifier was set to null
      * @throws IOException
      */
-    protected void checkRows(final boolean rowsExist) throws IOException {
+    protected void checkRows(final boolean rowsExist, final boolean qualifierNull) throws IOException {
         try (org.apache.hadoop.hbase.client.Table table = getDataContext().getConnection().getTable(TableName.valueOf(
                 TABLE_NAME))) {
             final Get get = new Get(Bytes.toBytes(RK_1));
             final Result result = table.get(get);
+
             if (rowsExist) {
                 assertFalse(result.isEmpty());
-                assertEquals(V_WORLD, new String(result.getValue(Bytes.toBytes(CF_FOO), Bytes.toBytes(Q_HELLO))));
-                assertEquals(V_THERE, new String(result.getValue(Bytes.toBytes(CF_FOO), Bytes.toBytes(Q_HI))));
-                assertEquals(V_YO, new String(result.getValue(Bytes.toBytes(CF_BAR), Bytes.toBytes(Q_HEY))));
-                assertEquals(V_123_BYTE_ARRAY.toString(), new String(result.getValue(Bytes.toBytes(CF_BAR), Bytes
-                        .toBytes(Q_BAH))));
+                if (!qualifierNull) {
+                    assertEquals(V_WORLD, new String(result.getValue(Bytes.toBytes(CF_FOO), Bytes.toBytes(Q_HELLO))));
+                    assertEquals(V_THERE, new String(result.getValue(Bytes.toBytes(CF_FOO), Bytes.toBytes(Q_HI))));
+                    assertEquals(V_YO, new String(result.getValue(Bytes.toBytes(CF_BAR), Bytes.toBytes(Q_HEY))));
+                    assertArrayEquals(V_123_BYTE_ARRAY, result.getValue(Bytes.toBytes(CF_BAR), Bytes.toBytes(Q_BAH)));
+                } else {
+                    assertEquals(V_THERE, new String(result.getValue(Bytes.toBytes(CF_FOO), null)));
+                    assertArrayEquals(V_123_BYTE_ARRAY, result.getValue(Bytes.toBytes(CF_BAR), null));
+                }
             } else {
                 assertTrue(result.isEmpty());
             }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/4a928d17/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
index a71422d..82ac9d8 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.*;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.List;
 
@@ -70,7 +71,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
             final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                     CF_BAR);
             final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
+                    CF_BAR, false);
             final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
             getUpdateCallback().insertInto(mutableTable, columns);
             fail("Should get an exception that the type of the table is wrong.");
@@ -114,24 +115,6 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     }
 
     /**
-     * Having the columns parameter empty at the builder, should throw an exception
-     *
-     * @throws IOException
-     */
-    @Test
-    public void testColumnsEmptyAtBuilder() throws IOException {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            List<HBaseColumn> emptyList = new ArrayList<>();
-            new HBaseRowInsertionBuilder(getUpdateCallback(), existingTable, emptyList);
-            fail("Should get an exception that the columns list is empty.");
-        } catch (IllegalArgumentException e) {
-            assertEquals("The hbaseColumns list is null or empty", e.getMessage());
-        }
-    }
-
-    /**
      * Using a table that doesn't exist in the schema, should throw an exception
      *
      * @throws IOException
@@ -144,7 +127,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
             final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                     CF_BAR);
             final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
+                    CF_BAR, false);
             final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
             getUpdateCallback().insertInto(wrongTable, columns);
             fail("Should get an exception that the table isn't in the schema.");
@@ -164,7 +147,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
         try {
             final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                     CF_BAR);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, null, CF_FOO, CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, null, CF_FOO, CF_BAR, false);
             final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
             getUpdateCallback().insertInto(existingTable, columns);
             fail("Should get an exception that ID-column doesn't exist.");
@@ -185,7 +168,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
             final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                     CF_BAR);
             final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
+                    CF_BAR, false);
             final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
             final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                     wrongColumnFamily, null);
@@ -209,7 +192,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
             final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                     CF_BAR);
             final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
+                    CF_BAR, false);
             final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
             final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
                     wrongColumnFamily);
@@ -228,7 +211,8 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     public void testCreatingTheHBaseClientWithTableNameNull() {
         try {
             final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
+                    false);
             final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
             final Object[] values = new String[] { "Values" };
             new HBaseClient(getDataContext().getConnection()).insertRow(null, columns, values, 0);
@@ -261,7 +245,8 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     public void testCreatingTheHBaseClientWithValuesNull() {
         try {
             final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
+                    false);
             final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
             new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, null, 0);
             fail("Should get an exception that values is null");
@@ -278,7 +263,8 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     public void testCreatingTheHBaseClientWithIndexOfIdColumnOutOfBounce() {
         try {
             final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
+                    false);
             final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
             final Object[] values = new String[] { "Values" };
             new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 10);
@@ -296,7 +282,8 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     public void testCreatingTheHBaseClientWithRowKeyNull() {
         try {
             final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
+                    false);
             final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
             final Object[] values = new String[] { null };
             new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 0);
@@ -308,6 +295,29 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     }
 
     /**
+     * Inserting a row without setting enough values directly on the HBaseClient, should throw exception.
+     * NOTE: This exception is already prevented when using the {@link HBaseRowInsertionBuilder}
+     * @throws IOException 
+     */
+    @Test
+    public void testNotSettingEnoughValues() throws IOException {
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR, false);
+            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+            final Collection<Object> values = getToLittleValues(row);
+            final HBaseClient hBaseClient = ((HBaseDataContext) getUpdateCallback().getDataContext()).getHBaseClient();
+            hBaseClient.insertRow(TABLE_NAME, columns.toArray(new HBaseColumn[columns.size()]), values.toArray(
+                    new Object[values.size()]), 0); // TODO: find the ID-column
+            fail("Should get an exception when insering directly into the HBaseClient without having enough values.");
+        } catch (IllegalArgumentException e) {
+            assertEquals("The amount of columns don't match the amount of values", e.getMessage());
+        }
+    }
+
+    /**
      * Goodflow. Using an existing table and columns, should work
      */
     @Test
@@ -316,7 +326,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
             final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                     CF_BAR);
             final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
+                    CF_BAR, false);
             final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
             getUpdateCallback().insertInto(existingTable, columns);
         } catch (Exception e) {
@@ -324,31 +334,27 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
         }
     }
 
-    // /**
-    // * Goodflow. Executing the insertInto with the Values being null, should not throw an exception
-    // */
-    // public void testNotSettingTheValues() {
-    // if (isConfigured()) {
-    // try {
-    // final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-    // CF_BAR);
-    // final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
-    // CF_FOO, CF_BAR);
-    // final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
-    //
-    // checkRows(false);
-    // final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable,
-    // columns);
-    // rowInsertionBuilder.execute();
-    // checkRows(false);
-    // } catch (Exception e) {
-    // fail("No exception should be thrown, when inserting without values.");
-    // }
-    // } else {
-    // warnAboutANotExecutedTest(getClass().getName(), new Object() {
-    // }.getClass().getEnclosingMethod().getName());
-    // }
-    // }
+    /**
+     * Goodflow, creating a row with qualifiers null should work.
+     */
+    @Test
+    public void testQaulifierNull() {
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR, true);
+            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+
+            checkRows(false, true);
+            final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
+            setValuesInInsertionBuilder(row, rowInsertionBuilder);
+            rowInsertionBuilder.execute();
+            checkRows(true, true);
+        } catch (Exception e) {
+            fail("Inserting a row without qualifiers should work.");
+        }
+    }
 
     /**
      * Goodflow. Inserting a row succesfully (with values set)
@@ -359,14 +365,14 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
             final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                     CF_BAR);
             final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
+                    CF_BAR, false);
             final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
 
-            checkRows(false);
+            checkRows(false, false);
             final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
             setValuesInInsertionBuilder(row, rowInsertionBuilder);
             rowInsertionBuilder.execute();
-            checkRows(true);
+            checkRows(true, false);
         } catch (Exception e) {
             fail("No exception should be thrown, when inserting with values.");
         }


[17/31] metamodel git commit: Refactored test so they no longer extends TestCase (and as such are no longer JUnit 3, but JUnit 4 tests) and they use "assumeTrue" to check if they should run or not. This reduces a lot of boiler plate code, because now we

Posted by ka...@apache.org.
Refactored test so they no longer extends TestCase (and as such are no longer JUnit 3, but JUnit 4 tests) and they use "assumeTrue" to check if they should run or not. This reduces a lot of boiler plate code, because now we don't need to call the "isConfigured" method in each test method.

Also added some extra null check in the code which cleans up after test, because some nullpointer exceptions were thrown when the tests were skipped.


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/717a3a44
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/717a3a44
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/717a3a44

Branch: refs/heads/master
Commit: 717a3a443624f2da1a883a26d17a25886c77d67e
Parents: f4d2c97
Author: Arjan Seijkens <a....@quadient.com>
Authored: Fri Jun 1 11:25:29 2018 +0200
Committer: Arjan Seijkens <a....@quadient.com>
Committed: Fri Jun 1 11:25:29 2018 +0200

----------------------------------------------------------------------
 .../apache/metamodel/hbase/CreateTableTest.java | 218 ++++-----
 .../apache/metamodel/hbase/DeleteRowTest.java   | 212 ++++-----
 .../apache/metamodel/hbase/DropTableTest.java   |  27 +-
 .../metamodel/hbase/HBaseDataContextTest.java   |  17 +-
 .../apache/metamodel/hbase/HBaseTestCase.java   |  34 +-
 .../hbase/HBaseUpdateCallbackTest.java          |  57 ++-
 .../apache/metamodel/hbase/InsertRowTest.java   | 444 ++++++++-----------
 7 files changed, 423 insertions(+), 586 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/717a3a44/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
index 6c39acd..b619403 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
@@ -18,6 +18,8 @@
  */
 package org.apache.metamodel.hbase;
 
+import static org.junit.Assert.*;
+
 import java.io.IOException;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
@@ -27,211 +29,169 @@ import java.util.stream.Collectors;
 
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.schema.ImmutableSchema;
+import org.junit.Test;
 
 public class CreateTableTest extends HBaseUpdateCallbackTest {
 
     /**
      * Check if creating table is supported
      */
+    @Test
     public void testCreateTableSupported() {
-        if (isConfigured()) {
-            assertTrue(getUpdateCallback().isCreateTableSupported());
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
-        }
+        assertTrue(getUpdateCallback().isCreateTableSupported());
     }
 
     /**
      * Create a table with an immutableSchema, should throw a IllegalArgumentException
      */
+    @Test
     public void testWrongSchema() {
-        if (isConfigured()) {
-            final ImmutableSchema immutableSchema = new ImmutableSchema(getSchema());
-            try {
-                getUpdateCallback().createTable(immutableSchema, TABLE_NAME).execute();
-                fail("Should get an exception that the schema isn't mutable");
-            } catch (IllegalArgumentException e) {
-                assertEquals("Not a mutable schema: " + immutableSchema, e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        final ImmutableSchema immutableSchema = new ImmutableSchema(getSchema());
+        try {
+            getUpdateCallback().createTable(immutableSchema, TABLE_NAME).execute();
+            fail("Should get an exception that the schema isn't mutable");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Not a mutable schema: " + immutableSchema, e.getMessage());
         }
     }
 
     /**
      * Create a table without columnFamilies, should throw a MetaModelException
      */
+    @Test
     public void testCreateTableWithoutColumnFamilies() {
-        if (isConfigured()) {
-            try {
-                getUpdateCallback().createTable(getSchema(), TABLE_NAME).execute();
-                fail("Should get an exception that the columnFamilies haven't been set");
-            } catch (MetaModelException e) {
-                assertEquals("Creating a table without columnFamilies", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            getUpdateCallback().createTable(getSchema(), TABLE_NAME).execute();
+            fail("Should get an exception that the columnFamilies haven't been set");
+        } catch (MetaModelException e) {
+            assertEquals("Creating a table without columnFamilies", e.getMessage());
         }
     }
 
     /**
      * Create a table with columnFamilies null, should throw a MetaModelException
      */
+    @Test
     public void testColumnFamiliesNull() {
-        if (isConfigured()) {
-            try {
-                getUpdateCallback().createTable(getSchema(), TABLE_NAME, null).execute();
-                fail("Should get an exception that the columnFamilies haven't been set");
-            } catch (MetaModelException e) {
-                assertEquals("Creating a table without columnFamilies", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            getUpdateCallback().createTable(getSchema(), TABLE_NAME, null).execute();
+            fail("Should get an exception that the columnFamilies haven't been set");
+        } catch (MetaModelException e) {
+            assertEquals("Creating a table without columnFamilies", e.getMessage());
         }
     }
 
     /**
      * Create a table with columnFamilies empty, should throw a MetaModelException
      */
+    @Test
     public void testColumnFamiliesEmpty() {
-        if (isConfigured()) {
-            try {
-                final LinkedHashSet<String> columnFamilies = new LinkedHashSet<String>();
-                getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
-                fail("Should get an exception that the columnFamilies haven't been set");
-            } catch (MetaModelException e) {
-                assertEquals("Creating a table without columnFamilies", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<String>();
+            getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
+            fail("Should get an exception that the columnFamilies haven't been set");
+        } catch (MetaModelException e) {
+            assertEquals("Creating a table without columnFamilies", e.getMessage());
         }
     }
 
     /**
      * Creating a HBaseClient with the tableName null, should throw a exception
      */
+    @Test
     public void testCreatingTheHBaseClientWithTableNameNull() {
-        if (isConfigured()) {
-            try {
-                final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
-                columnFamilies.add("1");
-                new HBaseClient(getDataContext().getConnection()).createTable(null, columnFamilies);
-                fail("Should get an exception that tableName is null");
-            } catch (IllegalArgumentException e) {
-                assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
+            columnFamilies.add("1");
+            new HBaseClient(getDataContext().getConnection()).createTable(null, columnFamilies);
+            fail("Should get an exception that tableName is null");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
         }
     }
 
     /**
      * Creating a HBaseClient with the tableName null, should throw a exception
      */
+    @Test
     public void testCreatingTheHBaseClientWithColumnFamiliesNull() {
-        if (isConfigured()) {
-            try {
-                new HBaseClient(getDataContext().getConnection()).createTable("1", null);
-                fail("Should get an exception that columnFamilies is null");
-            } catch (IllegalArgumentException e) {
-                assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            new HBaseClient(getDataContext().getConnection()).createTable("1", null);
+            fail("Should get an exception that columnFamilies is null");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
         }
     }
 
     /**
      * Creating a HBaseClient with the tableName null, should throw a exception
      */
+    @Test
     public void testCreatingTheHBaseClientWithColumnFamiliesEmpty() {
-        if (isConfigured()) {
-            try {
-                final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
-                new HBaseClient(getDataContext().getConnection()).createTable("1", columnFamilies);
-                fail("Should get an exception that columnFamilies is empty");
-            } catch (IllegalArgumentException e) {
-                assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
+            new HBaseClient(getDataContext().getConnection()).createTable("1", columnFamilies);
+            fail("Should get an exception that columnFamilies is empty");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
         }
     }
 
     /**
      * Goodflow. Create a table without the ID-Column, should work
-     * @throws IOException 
+     *
+     * @throws IOException
      */
+    @Test
     public void testCreateTableWithoutIDColumn() throws IOException {
-        if (isConfigured()) {
-            final HBaseTable table = createHBaseTable(TABLE_NAME, null, CF_FOO, CF_BAR, null);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, null, CF_FOO, CF_BAR);
-            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
-            try {
-                final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
-                        .createTable(getSchema(), TABLE_NAME);
-
-                hBaseCreateTableBuilder.setColumnFamilies(columnFamilies);
-                hBaseCreateTableBuilder.execute();
-                checkSuccesfullyInsertedTable();
-            } catch (Exception e) {
-                fail("Should not get an exception (that the ID-column is missing)");
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        final HBaseTable table = createHBaseTable(TABLE_NAME, null, CF_FOO, CF_BAR, null);
+        final LinkedHashMap<HBaseColumn, Object> row = createRow(table, null, CF_FOO, CF_BAR);
+        final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
+        try {
+            final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
+                    .createTable(getSchema(), TABLE_NAME);
+
+            hBaseCreateTableBuilder.setColumnFamilies(columnFamilies);
+            hBaseCreateTableBuilder.execute();
+            checkSuccesfullyInsertedTable();
+        } catch (Exception e) {
+            fail("Should not get an exception (that the ID-column is missing)");
         }
     }
 
     /**
      * Goodflow. Create a table including the ID-Column (columnFamilies not in constructor), should work
      */
+    @Test
     public void testSettingColumnFamiliesAfterConstrutor() {
-        if (isConfigured()) {
-            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
-            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
-            try {
-                final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
-                        .createTable(getSchema(), TABLE_NAME);
-
-                hBaseCreateTableBuilder.setColumnFamilies(columnFamilies);
-                hBaseCreateTableBuilder.execute();
-                checkSuccesfullyInsertedTable();
-            } catch (Exception e) {
-                fail("Should not get an exception");
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+        final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+        final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
+        try {
+            final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
+                    .createTable(getSchema(), TABLE_NAME);
+
+            hBaseCreateTableBuilder.setColumnFamilies(columnFamilies);
+            hBaseCreateTableBuilder.execute();
+            checkSuccesfullyInsertedTable();
+        } catch (Exception e) {
+            fail("Should not get an exception");
         }
     }
 
     /**
      * Goodflow. Create a table including the ID-Column (columnFamilies in constructor), should work
      */
+    @Test
     public void testCreateTableColumnFamiliesInConstrutor() {
-        if (isConfigured()) {
-            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
-            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
-            try {
-                getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
-                checkSuccesfullyInsertedTable();
-            } catch (Exception e) {
-                fail("Should not get an exception");
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+        final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+        final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
+        try {
+            getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
+            checkSuccesfullyInsertedTable();
+        } catch (Exception e) {
+            fail("Should not get an exception");
         }
     }
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/717a3a44/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
index ef594c8..3cf1ee4 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
@@ -18,196 +18,164 @@
  */
 package org.apache.metamodel.hbase;
 
+import static org.junit.Assert.*;
+
 import java.io.IOException;
 import java.util.LinkedHashMap;
 import java.util.List;
 
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.schema.MutableTable;
+import org.junit.Test;
 
 public class DeleteRowTest extends HBaseUpdateCallbackTest {
 
     /**
      * Delete is supported
      */
+    @Test
     public void testDeleteSupported() {
-        if (isConfigured()) {
-            assertTrue(getUpdateCallback().isDeleteSupported());
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
-        }
+        assertTrue(getUpdateCallback().isDeleteSupported());
     }
 
     /**
      * Having the table type wrong, should throw an exception
      */
+    @Test
     public void testTableWrongType() {
-        if (isConfigured()) {
-            final MutableTable mutableTable = new MutableTable();
-            try {
-                getUpdateCallback().deleteFrom(mutableTable);
-                fail("Should get an exception that the type of the table is wrong.");
-            } catch (IllegalArgumentException e) {
-                assertEquals("Not an HBase table: " + mutableTable, e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        final MutableTable mutableTable = new MutableTable();
+        try {
+            getUpdateCallback().deleteFrom(mutableTable);
+            fail("Should get an exception that the type of the table is wrong.");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Not an HBase table: " + mutableTable, e.getMessage());
         }
     }
 
     /**
      * Creating a HBaseRowDeletionBuilder with the hBaseClient null, should throw an exception
-     * @throws IOException 
+     *
+     * @throws IOException
      */
+    @Test
     public void testHBaseClientNullAtBuilder() throws IOException {
-        if (isConfigured()) {
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID,
-                        CF_FOO, CF_BAR);
-                new HBaseRowDeletionBuilder(null, existingTable);
-                fail("Should get an exception that hBaseClient can't be null.");
-            } catch (IllegalArgumentException e) {
-                assertEquals("hBaseClient cannot be null", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            new HBaseRowDeletionBuilder(null, existingTable);
+            fail("Should get an exception that hBaseClient can't be null.");
+        } catch (IllegalArgumentException e) {
+            assertEquals("hBaseClient cannot be null", e.getMessage());
         }
     }
 
     /**
      * Not setting the rowkey, should throw an exception
-     * @throws IOException 
+     *
+     * @throws IOException
      */
+    @Test
     public void testNotSettingRowkey() throws IOException {
-        if (isConfigured()) {
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID,
-                        CF_FOO, CF_BAR);
-                getUpdateCallback().deleteFrom(existingTable).execute();
-                fail("Should get an exception that the columnFamily doesn't exist.");
-            } catch (MetaModelException e) {
-                assertEquals("Key cannot be null", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            getUpdateCallback().deleteFrom(existingTable).execute();
+            fail("Should get an exception that the columnFamily doesn't exist.");
+        } catch (MetaModelException e) {
+            assertEquals("Key cannot be null", e.getMessage());
         }
     }
 
     /**
      * Creating a HBaseClient with the tableName null, should throw a exception
      */
+    @Test
     public void testCreatingTheHBaseClientWithTableNameNull() {
-        if (isConfigured()) {
-            try {
-                new HBaseClient(getDataContext().getConnection()).deleteRow(null, new String("1"));
-                fail("Should get an exception that tableName is null");
-            } catch (IllegalArgumentException e) {
-                assertEquals("Can't delete a row without having tableName or rowKey", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            new HBaseClient(getDataContext().getConnection()).deleteRow(null, new String("1"));
+            fail("Should get an exception that tableName is null");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't delete a row without having tableName or rowKey", e.getMessage());
         }
     }
 
     /**
      * Creating a HBaseClient with the rowKey null, should throw a exception
      */
+    @Test
     public void testCreatingTheHBaseClientWithRowKeyNull() {
-        if (isConfigured()) {
-            try {
-                new HBaseClient(getDataContext().getConnection()).deleteRow("tableName", null);
-                fail("Should get an exception that rowKey is null");
-            } catch (IllegalArgumentException e) {
-                assertEquals("Can't delete a row without having tableName or rowKey", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            new HBaseClient(getDataContext().getConnection()).deleteRow("tableName", null);
+            fail("Should get an exception that rowKey is null");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't delete a row without having tableName or rowKey", e.getMessage());
         }
     }
 
     /**
      * Goodflow. Deleting a row, that doesn't exist, should not throw an exception
      */
+    @Test
     public void testDeletingNotExistingRow() {
-        if (isConfigured()) {
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID,
-                        CF_FOO, CF_BAR);
-
-                checkRows(false);
-                final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback()
-                        .deleteFrom(existingTable);
-                rowDeletionBuilder.setKey(RK_1);
-                rowDeletionBuilder.execute();
-                checkRows(false);
-            } catch (Exception e) {
-                fail("Should not get an exception that the row doesn't exist.");
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+
+            checkRows(false);
+            final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
+                    existingTable);
+            rowDeletionBuilder.setKey(RK_1);
+            rowDeletionBuilder.execute();
+            checkRows(false);
+        } catch (Exception e) {
+            fail("Should not get an exception that the row doesn't exist.");
         }
     }
 
     /**
      * Goodflow. Deleting a row, which has an empty rowKey value, should not throw an exception
      */
+    @Test
     public void testUsingAnEmptyRowKeyValue() {
-        if (isConfigured()) {
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID,
-                        CF_FOO, CF_BAR);
-
-                checkRows(false);
-                final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback()
-                        .deleteFrom(existingTable);
-                rowDeletionBuilder.setKey("");
-                rowDeletionBuilder.execute();
-                checkRows(false);
-            } catch (Exception e) {
-                fail("Should not get an exception that the rowkey is empty.");
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+
+            checkRows(false);
+            final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
+                    existingTable);
+            rowDeletionBuilder.setKey("");
+            rowDeletionBuilder.execute();
+            checkRows(false);
+        } catch (Exception e) {
+            fail("Should not get an exception that the rowkey is empty.");
         }
     }
 
     /**
      * Goodflow. Deleting a row succesfully.
      */
+    @Test
     public void testDeleteRowSuccesfully() {
-        if (isConfigured()) {
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID,
-                        CF_FOO, CF_BAR);
-                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
-                        CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-
-                checkRows(false);
-                final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable,
-                        columns);
-                setValuesInInsertionBuilder(row, rowInsertionBuilder);
-                rowInsertionBuilder.execute();
-                checkRows(true);
-                final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback()
-                        .deleteFrom(existingTable);
-                rowDeletionBuilder.setKey(RK_1);
-                rowDeletionBuilder.execute();
-                checkRows(false);
-            } catch (Exception e) {
-                fail("Should not get an exception on deleting a row.");
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+
+            checkRows(false);
+            final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
+            setValuesInInsertionBuilder(row, rowInsertionBuilder);
+            rowInsertionBuilder.execute();
+            checkRows(true);
+            final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
+                    existingTable);
+            rowDeletionBuilder.setKey(RK_1);
+            rowDeletionBuilder.execute();
+            checkRows(false);
+        } catch (Exception e) {
+            fail("Should not get an exception on deleting a row.");
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/717a3a44/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
index afcc4eb..70ad3f8 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
@@ -18,31 +18,30 @@
  */
 package org.apache.metamodel.hbase;
 
+import static org.junit.Assert.*;
+
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.metamodel.MetaModelException;
+import org.junit.Test;
 
 public class DropTableTest extends HBaseUpdateCallbackTest {
 
     /**
      * Check if drop table is supported
      */
+    @Test
     public void testDropTableSupported() {
-        if (isConfigured()) {
             assertTrue(getUpdateCallback().isDropTableSupported());
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
-        }
     }
 
     /**
      * Trying to drop a table, that doesn't exist in the datastore, should throw a exception
      */
+    @Test
     public void testDropTableThatDoesntExist() {
-        if (isConfigured()) {
             try {
                 final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
                 getUpdateCallback().dropTable(table).execute();
@@ -50,35 +49,27 @@ public class DropTableTest extends HBaseUpdateCallbackTest {
             } catch (MetaModelException e) {
                 assertEquals("Trying to delete a table that doesn't exist in the datastore.", e.getMessage());
             }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
-        }
     }
 
     /**
      * Creating a HBaseClient with the tableName null, should throw a exception
      */
+    @Test
     public void testCreatingTheHBaseClientWithTableNameNull() {
-        if (isConfigured()) {
             try {
                 new HBaseClient(getDataContext().getConnection()).dropTable(null);
                 fail("Should get an exception that tableName is null");
             } catch (IllegalArgumentException e) {
                 assertEquals("Can't drop a table without having the tableName", e.getMessage());
             }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
-        }
     }
 
     /**
      * Goodflow. Droping a table succesfully.
      * @throws IOException
      */
+    @Test
     public void testDropTableSuccesfully() throws IOException {
-        if (isConfigured()) {
             try {
                 final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
@@ -89,9 +80,5 @@ public class DropTableTest extends HBaseUpdateCallbackTest {
             } catch (Exception e) {
                 fail("Should not get an exception that the table doesn't exist in the datastore");
             }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/717a3a44/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
index e4e647c..7d07e57 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
@@ -18,6 +18,8 @@
  */
 package org.apache.metamodel.hbase;
 
+import static org.junit.Assert.*;
+
 import java.io.IOException;
 import java.util.Arrays;
 
@@ -30,23 +32,20 @@ import org.apache.metamodel.data.DataSet;
 import org.apache.metamodel.schema.ColumnType;
 import org.apache.metamodel.schema.Table;
 import org.apache.metamodel.util.SimpleTableDef;
+import org.junit.Before;
+import org.junit.Test;
 
 public class HBaseDataContextTest extends HBaseTestCase {
 
     @Override
-    protected void setUp() throws Exception {
+    @Before
+    public void setUp() throws Exception {
         super.setUp();
-        if (isConfigured()) {
-            createTableNatively();
-        }
+        createTableNatively();
     }
 
+    @Test
     public void testCreateInsertQueryAndDrop() throws Exception {
-        if (!isConfigured()) {
-            System.err.println(getInvalidConfigurationMessage());
-            return;
-        }
-
         // test the schema exploration
         final Table table = getDataContext().getDefaultSchema().getTableByName(TABLE_NAME);
         assertNotNull(table);

http://git-wip-us.apache.org/repos/asf/metamodel/blob/717a3a44/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
index 5f494b7..785fdab 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
@@ -18,6 +18,8 @@
  */
 package org.apache.metamodel.hbase;
 
+import static org.junit.Assume.assumeTrue;
+
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
@@ -26,13 +28,11 @@ import java.util.Properties;
 import org.apache.metamodel.schema.ColumnType;
 import org.junit.AfterClass;
 
-import junit.framework.TestCase;
-
 /**
- * Properly configure before executing these tests. 
+ * Properly configure before executing these tests.
  * See the {@link HBaseTestCase#setUp()} and {@link HBaseTestCase#getPropertyFilePath()} methods.
  */
-public abstract class HBaseTestCase extends TestCase {
+public abstract class HBaseTestCase {
 
     // TableName
     protected static final String TABLE_NAME = "table_for_junit";
@@ -64,12 +64,10 @@ public abstract class HBaseTestCase extends TestCase {
 
     private String zookeeperHostname;
     private int zookeeperPort;
-    private boolean _configured;
     private static HBaseDataContext _dataContext;
 
-    @Override
     protected void setUp() throws Exception {
-        super.setUp();
+        boolean configured = false;
 
         Properties properties = new Properties();
         File file = new File(getPropertyFilePath());
@@ -81,20 +79,20 @@ public abstract class HBaseTestCase extends TestCase {
                 zookeeperPort = Integer.parseInt(zookeeperPortPropertyValue);
             }
 
-            _configured = (zookeeperHostname != null && !zookeeperHostname.isEmpty());
-        } else {
-            _configured = false;
-        }
-        if (isConfigured()) {
-            final HBaseConfiguration configuration = new HBaseConfiguration(zookeeperHostname, zookeeperPort,
-                    ColumnType.VARCHAR);
-            setDataContext(new HBaseDataContext(configuration));
+            configured = (zookeeperHostname != null && !zookeeperHostname.isEmpty());
         }
+        assumeTrue(configured);
+
+        final HBaseConfiguration configuration = new HBaseConfiguration(zookeeperHostname, zookeeperPort,
+                ColumnType.VARCHAR);
+        setDataContext(new HBaseDataContext(configuration));
     }
 
     @AfterClass
     public static void oneTimeTeardown() throws IOException {
-        _dataContext.getConnection().close();
+        if (_dataContext != null) {
+            _dataContext.getConnection().close();
+        }
     }
 
     /**
@@ -111,10 +109,6 @@ public abstract class HBaseTestCase extends TestCase {
                 + getPropertyFilePath() + "), to run integration tests";
     }
 
-    public boolean isConfigured() {
-        return _configured;
-    }
-
     public String getZookeeperHostname() {
         return zookeeperHostname;
     }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/717a3a44/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
index a30b7f2..b1e503f 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
@@ -18,6 +18,8 @@
  */
 package org.apache.metamodel.hbase;
 
+import static org.junit.Assert.*;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
@@ -33,6 +35,8 @@ import org.apache.metamodel.schema.ColumnType;
 import org.apache.metamodel.schema.MutableSchema;
 import org.apache.metamodel.schema.Table;
 import org.apache.metamodel.util.SimpleTableDef;
+import org.junit.After;
+import org.junit.Before;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,46 +47,37 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
     private HBaseUpdateCallback updateCallback;
     private MutableSchema schema;
 
-    private static boolean warningGiven = false;
-
     @Override
-    protected void setUp() throws Exception {
+    @Before
+    public void setUp() throws Exception {
         super.setUp();
-        if (isConfigured()) {
-            updateCallback = new HBaseUpdateCallback(getDataContext());
-            schema = (MutableSchema) getDataContext().getDefaultSchema();
-            dropTableIfItExists();
-        } else {
-            if (!warningGiven) {
-                System.err.println(getInvalidConfigurationMessage());
-                warningGiven = true;
-            }
-        }
+        updateCallback = new HBaseUpdateCallback(getDataContext());
+        schema = (MutableSchema) getDataContext().getDefaultSchema();
+        dropTableIfItExists();
     }
 
-    @Override
+    @After
     public void tearDown() throws Exception {
-        if (isConfigured()) {
-            dropTableIfItExists();
-        }
-        super.tearDown();
+        dropTableIfItExists();
     }
 
     /**
-     * Drop the table if it exists. 
+     * Drop the table if it exists.
      * After that check in the schema and the datastore if the actions have been executed succesfully.
      */
     protected void dropTableIfItExists() {
-        final Table table = schema.getTableByName(TABLE_NAME);
-        if (table != null) {
-            updateCallback.dropTable(table).execute();
-            // Check schema
-            assertNull(schema.getTableByName(TABLE_NAME));
-            // Check in the datastore
-            try (final Admin admin = getDataContext().getAdmin()) {
-                assertFalse(admin.tableExists(TableName.valueOf(TABLE_NAME)));
-            } catch (IOException e) {
-                fail("Should not an exception checking if the table exists");
+        if (schema != null) {
+            final Table table = schema.getTableByName(TABLE_NAME);
+            if (table != null) {
+                updateCallback.dropTable(table).execute();
+                // Check schema
+                assertNull(schema.getTableByName(TABLE_NAME));
+                // Check in the datastore
+                try (final Admin admin = getDataContext().getAdmin()) {
+                    assertFalse(admin.tableExists(TableName.valueOf(TABLE_NAME)));
+                } catch (IOException e) {
+                    fail("Should not an exception checking if the table exists");
+                }
             }
         }
     }
@@ -245,8 +240,8 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
 
     /**
      * Warn that the test(method) of a class is not executed, because the test-file hasn't been set.
-     * See {@link HBaseTestCase#getPropertyFilePath} 
-     * @param className 
+     * See {@link HBaseTestCase#getPropertyFilePath}
+     * @param className
      * @param methodName
      */
     protected void warnAboutANotExecutedTest(String className, String methodName) {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/717a3a44/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
index db04527..a71422d 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
@@ -18,6 +18,8 @@
  */
 package org.apache.metamodel.hbase;
 
+import static org.junit.Assert.*;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
@@ -25,362 +27,300 @@ import java.util.List;
 
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.schema.MutableTable;
+import org.junit.Test;
 
 public class InsertRowTest extends HBaseUpdateCallbackTest {
 
     /**
      * Check if inserting into a table is supported
+     *
      * @throws IOException
      */
+    @Test
     public void testInsertSupported() throws IOException {
-        if (isConfigured()) {
-            assertTrue(getUpdateCallback().isInsertSupported());
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
-        }
+        assertTrue(getUpdateCallback().isInsertSupported());
     }
 
     /**
      * Using only the table parameter, should throw an exception
+     *
      * @throws IOException
      */
+    @Test
     public void testOnlyUsingTableParameter() throws IOException {
-        if (isConfigured()) {
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                getUpdateCallback().insertInto(existingTable);
-                fail("Should get an exception that this method is not supported");
-            } catch (UnsupportedOperationException e) {
-                assertEquals("We need an explicit list of columns when inserting into an HBase table.", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            getUpdateCallback().insertInto(existingTable);
+            fail("Should get an exception that this method is not supported");
+        } catch (UnsupportedOperationException e) {
+            assertEquals("We need an explicit list of columns when inserting into an HBase table.", e.getMessage());
         }
     }
 
     /**
      * Having the table type wrong, should throw an exception
+     *
      * @throws IOException
      */
+    @Test
     public void testWrongTableType() throws IOException {
-        if (isConfigured()) {
-            final MutableTable mutableTable = new MutableTable();
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
-                        CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-                getUpdateCallback().insertInto(mutableTable, columns);
-                fail("Should get an exception that the type of the table is wrong.");
-            } catch (IllegalArgumentException e) {
-                assertEquals("Not an HBase table: " + mutableTable, e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        final MutableTable mutableTable = new MutableTable();
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+            getUpdateCallback().insertInto(mutableTable, columns);
+            fail("Should get an exception that the type of the table is wrong.");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Not an HBase table: " + mutableTable, e.getMessage());
         }
     }
 
     /**
      * Having the columns parameter null at the updateCallBack, should throw an exception
+     *
      * @throws IOException
      */
+    @Test
     public void testColumnsNullAtUpdateCallBack() throws IOException {
-        if (isConfigured()) {
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                getUpdateCallback().insertInto(existingTable, null);
-                fail("Should get an exception that the columns list is null.");
-            } catch (IllegalArgumentException e) {
-                assertEquals("The hbaseColumns list is null or empty", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            getUpdateCallback().insertInto(existingTable, null);
+            fail("Should get an exception that the columns list is null.");
+        } catch (IllegalArgumentException e) {
+            assertEquals("The hbaseColumns list is null or empty", e.getMessage());
         }
     }
 
     /**
      * Having the columns parameter empty at the updateCallBack, should throw an exception
+     *
      * @throws IOException
      */
+    @Test
     public void testColumnsEmptyAtUpdateCallBack() throws IOException {
-        if (isConfigured()) {
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                getUpdateCallback().insertInto(existingTable, new ArrayList<HBaseColumn>());
-                fail("Should get an exception that the columns list is empty.");
-            } catch (IllegalArgumentException e) {
-                assertEquals("The hbaseColumns list is null or empty", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            getUpdateCallback().insertInto(existingTable, new ArrayList<HBaseColumn>());
+            fail("Should get an exception that the columns list is empty.");
+        } catch (IllegalArgumentException e) {
+            assertEquals("The hbaseColumns list is null or empty", e.getMessage());
         }
     }
 
     /**
      * Having the columns parameter empty at the builder, should throw an exception
+     *
      * @throws IOException
      */
+    @Test
     public void testColumnsEmptyAtBuilder() throws IOException {
-        if (isConfigured()) {
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                List<HBaseColumn> emptyList = new ArrayList<>();
-                new HBaseRowInsertionBuilder(getUpdateCallback(), existingTable, emptyList);
-                fail("Should get an exception that the columns list is empty.");
-            } catch (IllegalArgumentException e) {
-                assertEquals("The hbaseColumns list is null or empty", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            List<HBaseColumn> emptyList = new ArrayList<>();
+            new HBaseRowInsertionBuilder(getUpdateCallback(), existingTable, emptyList);
+            fail("Should get an exception that the columns list is empty.");
+        } catch (IllegalArgumentException e) {
+            assertEquals("The hbaseColumns list is null or empty", e.getMessage());
         }
     }
 
     /**
      * Using a table that doesn't exist in the schema, should throw an exception
+     *
      * @throws IOException
      */
+    @Test
     public void testTableThatDoesntExist() throws IOException {
-        if (isConfigured()) {
-            final HBaseTable wrongTable = createHBaseTable("NewTableNotInSchema", HBaseDataContext.FIELD_ID, "cf1",
-                    "cf2", null);
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
-                        CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-                getUpdateCallback().insertInto(wrongTable, columns);
-                fail("Should get an exception that the table isn't in the schema.");
-            } catch (MetaModelException e) {
-                assertEquals("Trying to insert data into table: " + wrongTable.getName() + ", which doesn't exist yet",
-                        e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        final HBaseTable wrongTable = createHBaseTable("NewTableNotInSchema", HBaseDataContext.FIELD_ID, "cf1", "cf2",
+                null);
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+            getUpdateCallback().insertInto(wrongTable, columns);
+            fail("Should get an exception that the table isn't in the schema.");
+        } catch (MetaModelException e) {
+            assertEquals("Trying to insert data into table: " + wrongTable.getName() + ", which doesn't exist yet", e
+                    .getMessage());
         }
     }
 
     /**
      * If the ID-column doesn't exist in the columns array, then a exception should be thrown
+     *
      * @throws IOException
      */
+    @Test
     public void testIDColumnDoesntExistInColumnsArray() throws IOException {
-        if (isConfigured()) {
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, null, CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-                getUpdateCallback().insertInto(existingTable, columns);
-                fail("Should get an exception that ID-column doesn't exist.");
-            } catch (MetaModelException e) {
-                assertEquals("The ID-Column was not found", e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, null, CF_FOO, CF_BAR);
+            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+            getUpdateCallback().insertInto(existingTable, columns);
+            fail("Should get an exception that ID-column doesn't exist.");
+        } catch (MetaModelException e) {
+            assertEquals("The ID-Column was not found", e.getMessage());
         }
     }
 
     /**
      * If the column family doesn't exist in the table (wrong columnFamily), then a exception should be thrown
+     *
      * @throws IOException
      */
+    @Test
     public void testColumnFamilyDoesntExistsBecauseItsNull() throws IOException {
-        if (isConfigured()) {
-            final String wrongColumnFamily = "wrongColumnFamily";
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
-                        CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-                final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        wrongColumnFamily, null);
-                getUpdateCallback().insertInto(wrongTable, columns);
-                fail("Should get an exception that the columnFamily doesn't exist.");
-            } catch (MetaModelException e) {
-                assertEquals(String.format("ColumnFamily: %s doesn't exist in the schema of the table",
-                        wrongColumnFamily), e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        final String wrongColumnFamily = "wrongColumnFamily";
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+            final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    wrongColumnFamily, null);
+            getUpdateCallback().insertInto(wrongTable, columns);
+            fail("Should get an exception that the columnFamily doesn't exist.");
+        } catch (MetaModelException e) {
+            assertEquals(String.format("ColumnFamily: %s doesn't exist in the schema of the table", wrongColumnFamily),
+                    e.getMessage());
         }
     }
 
     /**
      * If the column family doesn't exist in the table (new columnFamily), then a exception should be thrown
+     *
      * @throws IOException
      */
+    @Test
     public void testColumnFamilyDoesntExistsBecauseItsNew() throws IOException {
-        if (isConfigured()) {
-            final String wrongColumnFamily = "newColumnFamily";
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
-                        CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-                final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
-                        wrongColumnFamily);
-                getUpdateCallback().insertInto(wrongTable, columns);
-                fail("Should get an exception that the columnFamily doesn't exist.");
-            } catch (MetaModelException e) {
-                assertEquals(String.format("ColumnFamily: %s doesn't exist in the schema of the table",
-                        wrongColumnFamily), e.getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        final String wrongColumnFamily = "newColumnFamily";
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+            final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
+                    wrongColumnFamily);
+            getUpdateCallback().insertInto(wrongTable, columns);
+            fail("Should get an exception that the columnFamily doesn't exist.");
+        } catch (MetaModelException e) {
+            assertEquals(String.format("ColumnFamily: %s doesn't exist in the schema of the table", wrongColumnFamily),
+                    e.getMessage());
         }
     }
 
     /**
      * Creating a HBaseClient with the tableName null, should throw a exception
      */
+    @Test
     public void testCreatingTheHBaseClientWithTableNameNull() {
-        if (isConfigured()) {
-            try {
-                final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-                final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
-                final Object[] values = new String[] { "Values" };
-                new HBaseClient(getDataContext().getConnection()).insertRow(null, columns, values, 0);
-                fail("Should get an exception that tableName is null");
-            } catch (IllegalArgumentException e) {
-                assertEquals(
-                        "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
-                                .getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+            final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
+            final Object[] values = new String[] { "Values" };
+            new HBaseClient(getDataContext().getConnection()).insertRow(null, columns, values, 0);
+            fail("Should get an exception that tableName is null");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
+                    .getMessage());
         }
     }
 
     /**
      * Creating a HBaseClient with the columns null, should throw a exception
      */
+    @Test
     public void testCreatingTheHBaseClientWithColumnsNull() {
-        if (isConfigured()) {
-            try {
-                final Object[] values = new String[] { "Values" };
-                new HBaseClient(getDataContext().getConnection()).insertRow("tableName", null, values, 0);
-                fail("Should get an exception that columns is null");
-            } catch (IllegalArgumentException e) {
-                assertEquals(
-                        "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
-                                .getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final Object[] values = new String[] { "Values" };
+            new HBaseClient(getDataContext().getConnection()).insertRow("tableName", null, values, 0);
+            fail("Should get an exception that columns is null");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
+                    .getMessage());
         }
     }
 
     /**
      * Creating a HBaseClient with the values null, should throw a exception
      */
+    @Test
     public void testCreatingTheHBaseClientWithValuesNull() {
-        if (isConfigured()) {
-            try {
-                final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-                final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
-                new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, null, 0);
-                fail("Should get an exception that values is null");
-            } catch (IllegalArgumentException e) {
-                assertEquals(
-                        "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
-                                .getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+            final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
+            new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, null, 0);
+            fail("Should get an exception that values is null");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
+                    .getMessage());
         }
     }
 
     /**
      * Creating a HBaseClient with the indexOfIdColumn out of bounce, should throw a exception
      */
+    @Test
     public void testCreatingTheHBaseClientWithIndexOfIdColumnOutOfBounce() {
-        if (isConfigured()) {
-            try {
-                final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-                final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
-                final Object[] values = new String[] { "Values" };
-                new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 10);
-                fail("Should get an exception that the indexOfIdColumn is incorrect");
-            } catch (IllegalArgumentException e) {
-                assertEquals(
-                        "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
-                                .getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+            final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
+            final Object[] values = new String[] { "Values" };
+            new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 10);
+            fail("Should get an exception that the indexOfIdColumn is incorrect");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
+                    .getMessage());
         }
     }
 
     /**
      * Creating a HBaseClient with the rowKey null, should throw a exception
      */
+    @Test
     public void testCreatingTheHBaseClientWithRowKeyNull() {
-        if (isConfigured()) {
-            try {
-                final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-                final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
-                final Object[] values = new String[] { null };
-                new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 0);
-                fail("Should get an exception that the indexOfIdColumn is incorrect");
-            } catch (IllegalArgumentException e) {
-                assertEquals(
-                        "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
-                                .getMessage());
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+            final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
+            final Object[] values = new String[] { null };
+            new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 0);
+            fail("Should get an exception that the indexOfIdColumn is incorrect");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
+                    .getMessage());
         }
     }
 
     /**
      * Goodflow. Using an existing table and columns, should work
      */
+    @Test
     public void testInsertIntoWithoutExecute() {
-        if (isConfigured()) {
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
-                        CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-                getUpdateCallback().insertInto(existingTable, columns);
-            } catch (Exception e) {
-                fail("No exception should be thrown, when inserting into an existing table.");
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+            getUpdateCallback().insertInto(existingTable, columns);
+        } catch (Exception e) {
+            fail("No exception should be thrown, when inserting into an existing table.");
         }
     }
 
@@ -413,27 +353,22 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     /**
      * Goodflow. Inserting a row succesfully (with values set)
      */
+    @Test
     public void testInsertingSuccesfully() {
-        if (isConfigured()) {
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
-                        CF_FOO, CF_BAR);
-                final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+        try {
+            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
+                    CF_BAR);
+            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
 
-                checkRows(false);
-                final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable,
-                        columns);
-                setValuesInInsertionBuilder(row, rowInsertionBuilder);
-                rowInsertionBuilder.execute();
-                checkRows(true);
-            } catch (Exception e) {
-                fail("No exception should be thrown, when inserting with values.");
-            }
-        } else {
-            warnAboutANotExecutedTest(getClass().getName(), new Object() {
-            }.getClass().getEnclosingMethod().getName());
+            checkRows(false);
+            final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
+            setValuesInInsertionBuilder(row, rowInsertionBuilder);
+            rowInsertionBuilder.execute();
+            checkRows(true);
+        } catch (Exception e) {
+            fail("No exception should be thrown, when inserting with values.");
         }
     }
 
@@ -446,5 +381,4 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     private static HBaseColumn[] convertToHBaseColumnsArray(List<HBaseColumn> columns) {
         return columns.toArray(new HBaseColumn[columns.size()]);
     }
-
 }


[08/31] metamodel git commit: Refactored HBaseRowInsertionBuilder, so it uses the _columns of the AbstractRowBuilder instead of its own, to make sure the model stays consistent.

Posted by ka...@apache.org.
Refactored HBaseRowInsertionBuilder, so it uses the _columns of the AbstractRowBuilder instead of its own, to make sure the model stays consistent.


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/ea7356a4
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/ea7356a4
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/ea7356a4

Branch: refs/heads/master
Commit: ea7356a4228f700b1b8f7115762cfc42ca6c92fc
Parents: 3ada7eb
Author: arjansh <Ar...@humaninference.com>
Authored: Fri May 25 12:08:05 2018 +0200
Committer: arjansh <Ar...@humaninference.com>
Committed: Fri May 25 12:08:05 2018 +0200

----------------------------------------------------------------------
 .../insert/AbstractRowInsertionBuilder.java     |  6 +++
 .../hbase/HBaseRowInsertionBuilder.java         | 39 +++++++-------------
 .../metamodel/hbase/HBaseUpdateCallback.java    | 16 +++++++-
 3 files changed, 34 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/ea7356a4/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java b/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
index 58c0da8..bf6dcd9 100644
--- a/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
+++ b/core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java
@@ -44,6 +44,12 @@ public abstract class AbstractRowInsertionBuilder<U extends UpdateCallback> exte
         _table = table;
     }
 
+    public AbstractRowInsertionBuilder(final U updateCallback, final Table table, final List<Column> columns) {
+        super(columns);
+        _updateCallback = updateCallback;
+        _table = table;
+    }
+
     @Override
     public Table getTable() {
         return _table;

http://git-wip-us.apache.org/repos/asf/metamodel/blob/ea7356a4/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
index 1c633f7..7272f76 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
@@ -19,21 +19,20 @@
 package org.apache.metamodel.hbase;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Set;
+import java.util.Arrays;
+import java.util.List;
 
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.insert.AbstractRowInsertionBuilder;
+import org.apache.metamodel.schema.Column;
 
 /**
  * A builder-class to insert rows in a HBase datastore
  */
 public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
-
-    private HBaseColumn[] _hbaseColumns;
-
-    public HBaseRowInsertionBuilder(final HBaseUpdateCallback updateCallback, final HBaseTable table) {
-        super(updateCallback, table);
+    public HBaseRowInsertionBuilder(final HBaseUpdateCallback updateCallback, final HBaseTable table,
+            final List<Column> columns) {
+        super(updateCallback, table, columns);
         checkTable(updateCallback, table);
     }
 
@@ -53,25 +52,9 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
         tableInSchema.checkForNotMatchingColumns(tableGettingInserts.getColumnNames());
     }
 
-    /**
-     * Set the hbaseColumns. Checks if the columnFamilies exist in the table.
-     * @param hbaseColumns a {@link IllegalArgumentException} is thrown when this parameter is null or empty
-     */
-    public void setHbaseColumns(HBaseColumn[] hbaseColumns) {
-        if (hbaseColumns == null || hbaseColumns.length == 0) {
-            throw new IllegalArgumentException("List of hbaseColumns is null or empty");
-        }
-        final Set<String> columnFamilies = HBaseColumn.getColumnFamilies(hbaseColumns);
-        final HBaseTable tableInSchema = (HBaseTable) getTable();
-        final ArrayList<String> columnFamiliesAsList = new ArrayList<String>();
-        columnFamiliesAsList.addAll(columnFamilies);
-        tableInSchema.checkForNotMatchingColumns(columnFamiliesAsList);
-        this._hbaseColumns = hbaseColumns;
-    }
-
     @Override
     public synchronized void execute() {
-        if (_hbaseColumns == null || _hbaseColumns.length == 0) {
+        if (getColumns() == null || getColumns().length == 0) {
             throw new MetaModelException("The hbaseColumns-array is null or empty");
         }
         if (getValues() == null || getValues().length == 0) {
@@ -79,9 +62,15 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
         }
         try {
             final HBaseClient hBaseClient = getUpdateCallback().getHBaseClient();
-            hBaseClient.writeRow((HBaseTable) getTable(), _hbaseColumns, getValues());
+            hBaseClient.writeRow((HBaseTable) getTable(), getColumns(), getValues());
         } catch (IOException e) {
             throw new MetaModelException(e);
         }
     }
+
+    @Override
+    public HBaseColumn[] getColumns() {
+        return Arrays.stream(super.getColumns()).map(column -> (HBaseColumn) column).toArray(
+                size -> new HBaseColumn[size]);
+    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/ea7356a4/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
index e548229..6d35508 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
@@ -18,7 +18,9 @@
  */
 package org.apache.metamodel.hbase;
 
+import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.metamodel.AbstractUpdateCallback;
 import org.apache.metamodel.UpdateCallback;
@@ -26,6 +28,7 @@ import org.apache.metamodel.create.TableCreationBuilder;
 import org.apache.metamodel.delete.RowDeletionBuilder;
 import org.apache.metamodel.drop.TableDropBuilder;
 import org.apache.metamodel.insert.RowInsertionBuilder;
+import org.apache.metamodel.schema.Column;
 import org.apache.metamodel.schema.Schema;
 import org.apache.metamodel.schema.Table;
 
@@ -68,9 +71,18 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
     }
 
     @Override
-    public RowInsertionBuilder insertInto(Table table) {
+    public RowInsertionBuilder insertInto(final Table table) {
+        throw new UnsupportedOperationException(
+                "We need an explicit list of columns when inserting into an HBase table.");
+    }
+
+    public RowInsertionBuilder insertInto(final Table table, final List<HBaseColumn> columns) {
         if (table instanceof HBaseTable) {
-            return new HBaseRowInsertionBuilder(this, (HBaseTable) table);
+            return new HBaseRowInsertionBuilder(this, (HBaseTable) table, columns
+                    .stream()
+                    .map(obj -> (Column) obj)
+                    .collect(
+                    Collectors.toList()));
         } else {
             throw new IllegalArgumentException("Not an HBase table: " + table);
         }


[13/31] metamodel git commit: - Where missing added final qualifiers to method parameters. - Moved static methods around to more logic locations. From my point of view the HBaseColumn class is not a Helper class to provide those. - Changed the signature

Posted by ka...@apache.org.
- Where missing added final qualifiers to method parameters.
- Moved static methods around to more logic locations. From my point of view the HBaseColumn class is not a Helper class to provide those.
- Changed the signature of the HBaseRowInsertionBuilder constructor, so it take a List of HBaseColumn object as a parameter, this makes it possible to remove a few of the Helper methods which cast List<HBaseColumn> to List<Column> and vice versa.
- Refactored the logic which looks up the id column in an array or list.
- Removed the "public" getHBaseColumnsInternal() method from HBaseTable, which essentially was only invoked in combination with a static getColumnFamilies method. Instead I added a getColumnFamilies() method to the HBaseTable class which return the column families for that HBase table.


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/5a28d73c
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/5a28d73c
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/5a28d73c

Branch: refs/heads/master
Commit: 5a28d73cdddfc8b11ca2478e390c8c94bcf34310
Parents: 54a900b
Author: Arjan Seijkens <a....@quadient.com>
Authored: Thu May 31 10:38:13 2018 +0200
Committer: Arjan Seijkens <a....@quadient.com>
Committed: Thu May 31 10:38:13 2018 +0200

----------------------------------------------------------------------
 .../org/apache/metamodel/hbase/HBaseColumn.java | 108 -------------------
 .../hbase/HBaseRowInsertionBuilder.java         |  79 +++++++++++---
 .../org/apache/metamodel/hbase/HBaseTable.java  |  76 +++++++------
 .../metamodel/hbase/HBaseUpdateCallback.java    |  17 +--
 .../apache/metamodel/hbase/CreateTableTest.java |  18 +++-
 .../apache/metamodel/hbase/InsertRowTest.java   |  22 ++--
 6 files changed, 144 insertions(+), 176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/5a28d73c/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
index d6554c4..4470e10 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
@@ -18,14 +18,6 @@
  */
 package org.apache.metamodel.hbase;
 
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.metamodel.schema.Column;
 import org.apache.metamodel.schema.ColumnType;
 import org.apache.metamodel.schema.ColumnTypeImpl;
 import org.apache.metamodel.schema.MutableColumn;
@@ -128,104 +120,4 @@ public final class HBaseColumn extends MutableColumn {
     public String getQuote() {
         return null;
     }
-
-    /**
-     * Creates a set of columnFamilies out of a list of hbaseColumns
-     * @param columns
-     * @return {@link LinkedHashSet}
-     */
-    public static Set<String> getColumnFamilies(List<HBaseColumn> columns) {
-        final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
-        for (HBaseColumn column : columns) {
-            columnFamilies.add(column.getColumnFamily());
-        }
-        return columnFamilies;
-    }
-
-    /**
-     * Returns the index of the ID-column (see {@link HBaseDataContext#FIELD_ID}) in an array of HBaseColumns.
-     * When no ID-column is found, then null is returned.
-     * @param columns
-     * @return {@link Integer}
-     */
-    public static Integer findIndexOfIdColumn(List<HBaseColumn> columns) {
-        int i = 0;
-        Integer indexOfIDColumn = null;
-        Iterator<HBaseColumn> iterator = columns.iterator();
-        while (indexOfIDColumn == null && iterator.hasNext()) {
-            indexOfIDColumn = findIndexOfIdColumn(iterator.next().getColumnFamily(), i);
-            if (indexOfIDColumn == null) {
-                i++;
-            }
-        }
-        return indexOfIDColumn;
-    }
-
-    /**
-     * Returns the index of the ID-column (see {@link HBaseDataContext#FIELD_ID}) in an array of columnNames.
-     * When no ID-column is found, then null is returned.
-     * @param columnNames
-     * @return {@link Integer}
-     */
-    public static Integer findIndexOfIdColumn(String[] columnNames) {
-        int i = 0;
-        Integer indexOfIDColumn = null;
-        while (indexOfIDColumn == null && i < columnNames.length) {
-            indexOfIDColumn = findIndexOfIdColumn(columnNames[i], i);
-            if (indexOfIDColumn == null) {
-                i++;
-            }
-        }
-        return indexOfIDColumn;
-    }
-
-    /**
-     * Returns the index of the ID-column (see {@link HBaseDataContext#FIELD_ID})
-     * When no ID-column is found, then null is returned.
-     * @param columnNames
-     * @return {@link Integer}
-     */
-    private static Integer findIndexOfIdColumn(String columnName, int index) {
-        Integer indexOfIDColumn = null;
-        if (columnName.equals(HBaseDataContext.FIELD_ID)) {
-            indexOfIDColumn = new Integer(index);
-        }
-        return indexOfIDColumn;
-    }
-
-    /**
-     * Converts a list of {@link Column}'s to a list of {@link HBaseColumn}'s
-     * @param columns
-     * @return {@link List}<{@link HBaseColumn}>
-     */
-    public static List<HBaseColumn> convertToHBaseColumnsList(List<Column> columns) {
-        return columns.stream().map(column -> (HBaseColumn) column).collect(Collectors.toList());
-    }
-
-    /**
-     * Converts a list of {@link HBaseColumn}'s to a list of {@link Column}'s
-     * @param columns
-     * @return {@link List}<{@link Column}>
-     */
-    public static List<Column> convertToColumnsList(List<HBaseColumn> columns) {
-        return columns.stream().map(column -> (Column) column).collect(Collectors.toList());
-    }
-
-    /**
-     * Converts a list of {@link HBaseColumn}'s to an array of {@link HBaseColumn}'s
-     * @param columns
-     * @return Array of {@link HBaseColumn}
-     */
-    public static HBaseColumn[] convertToHBaseColumnsArray(List<HBaseColumn> columns) {
-        return columns.stream().map(column -> column).toArray(size -> new HBaseColumn[size]);
-    }
-
-    /**
-     * Converts a array of {@link Column}'s to an array of {@link HBaseColumn}'s
-     * @param columns
-     * @return Array of {@link HBaseColumn}
-     */
-    public static HBaseColumn[] convertToHBaseColumnsArray(Column[] columns) {
-        return Arrays.stream(columns).map(column -> (HBaseColumn) column).toArray(size -> new HBaseColumn[size]);
-    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5a28d73c/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
index 9e36a4d..9091788 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
@@ -18,7 +18,12 @@
  */
 package org.apache.metamodel.hbase;
 
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
 import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.insert.AbstractRowInsertionBuilder;
@@ -30,7 +35,7 @@ import org.apache.metamodel.schema.Column;
 // TODO: Possible future improvement: Make it possible to change the columns for each execute.
 // Now each row will get exactly the same columns.
 public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
-    final Integer _indexOfIdColumn;
+    final int _indexOfIdColumn;
 
     /**
      * Creates a {@link HBaseRowInsertionBuilder}. The table and the column's columnFamilies are checked to exist in the schema.
@@ -41,25 +46,41 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
      * @throws MetaModelException when no ID-column is found.
      */
     public HBaseRowInsertionBuilder(final HBaseUpdateCallback updateCallback, final HBaseTable table,
-            final List<Column> columns) {
-        super(updateCallback, table, columns);
+            final List<HBaseColumn> columns) {
+        super(updateCallback, table, columns.stream().map(column -> (Column) column).collect(Collectors.toList()));
         if (columns.isEmpty()) { // TODO: Columns null will already result in a NullPointer at the super. Should the
                                  // super get a extra check?
             throw new IllegalArgumentException("The hbaseColumns list is null or empty");
         }
 
-        this._indexOfIdColumn = HBaseColumn.findIndexOfIdColumn(HBaseColumn.convertToHBaseColumnsList(columns));
-        if (_indexOfIdColumn == null) {
+        this._indexOfIdColumn = getIndexOfIdColumn(columns);
+        if (_indexOfIdColumn == -1) {
             throw new MetaModelException("The ID-Column was not found");
         }
 
         checkTable(updateCallback, table);
-        table.checkForNotMatchingColumnFamilies(HBaseColumn.getColumnFamilies(HBaseColumn.convertToHBaseColumnsList(
-                columns)));
+        checkForNotMatchingColumnFamilies(table, getColumnFamilies(columns));
+    }
+
+    /**
+     * Returns the index of the ID-column (see {@link HBaseDataContext#FIELD_ID}) in an array of HBaseColumns. When no
+     * ID-column is found, then null is returned.
+     *
+     * @param columns
+     * @return {@link Integer}
+     */
+    private static Integer getIndexOfIdColumn(final List<HBaseColumn> columns) {
+        for (int i = 0; i < columns.size(); i++) {
+            if (HBaseDataContext.FIELD_ID.equals(columns.get(i).getColumnFamily())) {
+                return i;
+            }
+        }
+        return -1;
     }
 
     /**
      * Check if the table and it's columnFamilies exist in the schema
+     *
      * @param updateCallback
      * @param tableGettingInserts
      * @throws MetaModelException If the table or the columnFamilies don't exist
@@ -71,18 +92,52 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
             throw new MetaModelException("Trying to insert data into table: " + tableGettingInserts.getName()
                     + ", which doesn't exist yet");
         }
-        tableInSchema.checkForNotMatchingColumnFamilies(HBaseColumn.getColumnFamilies(tableGettingInserts
-                .getHBaseColumnsInternal()));
+        checkForNotMatchingColumnFamilies(tableInSchema, tableGettingInserts.getColumnFamilies());
+    }
+
+    /**
+     * Check if a list of columnNames all exist in this table
+     *
+     * @param columnNamesOfCheckedTable
+     * @throws MetaModelException If a column doesn't exist
+     */
+    public void checkForNotMatchingColumnFamilies(final HBaseTable table, final Set<String> columnNamesOfCheckedTable) {
+        Set<String> columnFamilyNamesOfExistingTable = table.getColumnFamilies();
+
+        for (String columnNameOfCheckedTable : columnNamesOfCheckedTable) {
+            boolean matchingColumnFound = false;
+            Iterator<String> iterator = columnFamilyNamesOfExistingTable.iterator();
+            while (!matchingColumnFound && iterator.hasNext()) {
+                if (columnNameOfCheckedTable.equals(iterator.next())) {
+                    matchingColumnFound = true;
+                }
+            }
+            if (!matchingColumnFound) {
+                throw new MetaModelException(String.format("ColumnFamily: %s doesn't exist in the schema of the table",
+                        columnNameOfCheckedTable));
+            }
+        }
+    }
+
+    /**
+     * Creates a set of columnFamilies out of a list of hbaseColumns
+     *
+     * @param columns
+     * @return {@link LinkedHashSet}
+     */
+    private static Set<String> getColumnFamilies(final List<HBaseColumn> columns) {
+        return columns.stream().map(HBaseColumn::getColumnFamily).distinct().collect(Collectors.toSet());
     }
 
     @Override
     public synchronized void execute() {
-        getUpdateCallback().getHBaseClient().insertRow(getTable().getName(), getColumns(), getValues(), _indexOfIdColumn
-                .intValue());
+        getUpdateCallback().getHBaseClient().insertRow(getTable().getName(), getColumns(), getValues(),
+                _indexOfIdColumn);
     }
 
     @Override
     public HBaseColumn[] getColumns() {
-        return HBaseColumn.convertToHBaseColumnsArray(super.getColumns());
+        return Arrays.stream(super.getColumns()).map(column -> (HBaseColumn) column).toArray(
+                size -> new HBaseColumn[size]);
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5a28d73c/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
index 9f482ac..0dd2558 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
@@ -18,9 +18,9 @@
  */
 package org.apache.metamodel.hbase;
 
-import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.metamodel.MetaModelException;
@@ -45,16 +45,16 @@ final class HBaseTable extends MutableTable {
     private final transient ColumnType _defaultRowKeyColumnType;
 
     /**
-     * Creates an HBaseTable. If the tableDef variable doesn't include the ID-column (see {@link HBaseDataContext#FIELD_ID}). 
+     * Creates an HBaseTable. If the tableDef variable doesn't include the ID-column (see {@link HBaseDataContext#FIELD_ID}).
      * Then it's first added.
      * @param dataContext
      * @param tableDef Table definition. The tableName, columnNames and columnTypes variables are used.
      * @param schema {@link MutableSchema} where the table belongs to.
-     * @param defaultRowKeyColumnType This variable determines the {@link ColumnType}, 
-     * used when the tableDef doesn't include the ID column (see {@link HBaseDataContext#FIELD_ID}). 
+     * @param defaultRowKeyColumnType This variable determines the {@link ColumnType},
+     * used when the tableDef doesn't include the ID column (see {@link HBaseDataContext#FIELD_ID}).
      */
-    public HBaseTable(HBaseDataContext dataContext, SimpleTableDef tableDef, MutableSchema schema,
-            ColumnType defaultRowKeyColumnType) {
+    public HBaseTable(final HBaseDataContext dataContext, final SimpleTableDef tableDef, final MutableSchema schema,
+            final ColumnType defaultRowKeyColumnType) {
         super(tableDef.getName(), TableType.TABLE, schema);
         _dataContext = dataContext;
         _defaultRowKeyColumnType = defaultRowKeyColumnType;
@@ -66,7 +66,7 @@ final class HBaseTable extends MutableTable {
      * @param tableDef
      * @param defaultRowKeyColumnType
      */
-    private void addColumns(SimpleTableDef tableDef, ColumnType defaultRowKeyColumnType) {
+    private void addColumns(final SimpleTableDef tableDef, final ColumnType defaultRowKeyColumnType) {
         // Add the columns
         final String[] columnNames = tableDef.getColumnNames();
         if (columnNames == null || columnNames.length == 0) {
@@ -75,13 +75,12 @@ final class HBaseTable extends MutableTable {
             final ColumnType[] columnTypes = tableDef.getColumnTypes();
 
             // Find the ID-Column
-            Integer indexOfIDColumn = HBaseColumn.findIndexOfIdColumn(columnNames);
-            boolean idColumnFound = indexOfIDColumn != null;
+            int indexOfIDColumn = getIndexOfIdColumn(columnNames);
+            boolean idColumnFound = indexOfIDColumn != -1;
 
             // ColumnNumbers start from 1
             if (idColumnFound) {
-                addColumn(HBaseDataContext.FIELD_ID, columnTypes[indexOfIDColumn.intValue()], indexOfIDColumn.intValue()
-                        + 1);
+                addColumn(HBaseDataContext.FIELD_ID, columnTypes[indexOfIDColumn], indexOfIDColumn + 1);
             } else {
                 addColumn(HBaseDataContext.FIELD_ID, defaultRowKeyColumnType, 1);
             }
@@ -100,7 +99,24 @@ final class HBaseTable extends MutableTable {
     }
 
     /**
+     * Returns the index of the ID-column (see {@link HBaseDataContext#FIELD_ID}) in an array of columnNames. When no
+     * ID-column is found, then -1 is returned.
+     *
+     * @param columnNames
+     * @return {@link Integer}
+     */
+    private static int getIndexOfIdColumn(final String[] columnNames) {
+        for (int i = 0; i < columnNames.length; i++) {
+            if (HBaseDataContext.FIELD_ID.equals(columnNames[i])) {
+                return i;
+            }
+        }
+        return -1;
+    }
+
+    /**
      * Add a column to this table
+     *
      * @param columnName
      * @param columnType
      * @param columnNumber
@@ -133,34 +149,16 @@ final class HBaseTable extends MutableTable {
     }
 
     /**
-     * Check if a list of columnNames all exist in this table
-     * @param columnNamesOfCheckedTable
-     * @throws MetaModelException If a column doesn't exist
-     */
-    public void checkForNotMatchingColumnFamilies(final Set<String> columnNamesOfCheckedTable) {
-        Set<String> columnFamilyNamesOfExistingTable = HBaseColumn.getColumnFamilies(getHBaseColumnsInternal());
-
-        for (String columnNameOfCheckedTable : columnNamesOfCheckedTable) {
-            boolean matchingColumnFound = false;
-            Iterator<String> iterator = columnFamilyNamesOfExistingTable.iterator();
-            while (!matchingColumnFound && iterator.hasNext()) {
-                if (columnNameOfCheckedTable.equals(iterator.next())) {
-                    matchingColumnFound = true;
-                }
-            }
-            if (!matchingColumnFound) {
-                throw new MetaModelException(String.format("ColumnFamily: %s doesn't exist in the schema of the table",
-                        columnNameOfCheckedTable));
-            }
-        }
-    }
-
-    /**
-     * Returns a list of {@link HBaseColumn}'s from {@link HBaseTable#getColumnsInternal()}, 
-     * which returns a list of {@link Column}'s
-     * @return {@link List}<{@link HBaseColumn}>
+     * Returns the column families for this HBase table.
+     *
+     * @return {@link Set}
      */
-    public List<HBaseColumn> getHBaseColumnsInternal() {
-        return HBaseColumn.convertToHBaseColumnsList(getColumnsInternal());
+    Set<String> getColumnFamilies() {
+        return getColumnsInternal()
+                .stream()
+                .map(column -> (HBaseColumn) column)
+                .map(HBaseColumn::getColumnFamily)
+                .distinct()
+                .collect(Collectors.toSet());
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5a28d73c/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
index b154275..9273b07 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
@@ -37,13 +37,13 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
 
     private final HBaseClient _hBaseClient;
 
-    public HBaseUpdateCallback(HBaseDataContext dataContext) {
+    public HBaseUpdateCallback(final HBaseDataContext dataContext) {
         super(dataContext);
         _hBaseClient = new HBaseClient(dataContext.getConnection());
     }
 
     @Override
-    public TableCreationBuilder createTable(Schema schema, String name) {
+    public TableCreationBuilder createTable(final Schema schema, final String name) {
         return new HBaseCreateTableBuilder(this, schema, name);
     }
 
@@ -54,7 +54,8 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
      * @param columnFamilies the columnFamilies of the new table
      * @return {@link HBaseCreateTableBuilder}
      */
-    public HBaseCreateTableBuilder createTable(Schema schema, String name, Set<String> columnFamilies) {
+    public HBaseCreateTableBuilder createTable(final Schema schema, final String name,
+            final Set<String> columnFamilies) {
         return new HBaseCreateTableBuilder(this, schema, name, columnFamilies);
     }
 
@@ -64,7 +65,7 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
     }
 
     @Override
-    public TableDropBuilder dropTable(Table table) {
+    public TableDropBuilder dropTable(final Table table) {
         return new HBaseTableDropBuilder(table, this);
     }
 
@@ -78,7 +79,7 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
     }
 
     /**
-     * Initiates the building of a row insertion operation. 
+     * Initiates the building of a row insertion operation.
      * @param table Table to get inserts.
      * @param columns List of {@link HBaseColumn} to insert on.
      * @return {@link HBaseRowInsertionBuilder}
@@ -89,7 +90,7 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
             throw new IllegalArgumentException("The hbaseColumns list is null or empty");
         }
         if (table instanceof HBaseTable) {
-            return new HBaseRowInsertionBuilder(this, (HBaseTable) table, HBaseColumn.convertToColumnsList(columns));
+            return new HBaseRowInsertionBuilder(this, (HBaseTable) table, columns);
         } else {
             throw new IllegalArgumentException("Not an HBase table: " + table);
         }
@@ -104,9 +105,9 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
      * @throws IllegalArgumentException when table isn't a {@link HBaseTable}
      */
     @Override
-    public RowDeletionBuilder deleteFrom(Table table) {
+    public RowDeletionBuilder deleteFrom(final Table table) {
         if (table instanceof HBaseTable) {
-            return new HBaseRowDeletionBuilder(_hBaseClient, (HBaseTable) table);
+            return new HBaseRowDeletionBuilder(_hBaseClient, table);
         } else {
             throw new IllegalArgumentException("Not an HBase table: " + table);
         }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5a28d73c/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
index 5655e4a..9ea04df 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
@@ -20,7 +20,9 @@ package org.apache.metamodel.hbase;
 
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
+import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.schema.ImmutableSchema;
@@ -116,7 +118,7 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
         if (isConfigured()) {
             final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
             final LinkedHashMap<HBaseColumn, Object> row = createRow(table, null, CF_FOO, CF_BAR);
-            final Set<String> columnFamilies = HBaseColumn.getColumnFamilies(getHBaseColumnsFromMap(row));
+            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromMap(row));
             try {
                 final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
                         .createTable(getSchema(), TABLE_NAME);
@@ -194,7 +196,7 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
         if (isConfigured()) {
             final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
             final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
-            final Set<String> columnFamilies = HBaseColumn.getColumnFamilies(getHBaseColumnsFromMap(row));
+            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromMap(row));
             try {
                 final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
                         .createTable(getSchema(), TABLE_NAME);
@@ -218,7 +220,7 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
         if (isConfigured()) {
             final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
             final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
-            final Set<String> columnFamilies = HBaseColumn.getColumnFamilies(getHBaseColumnsFromMap(row));
+            final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromMap(row));
             try {
                 getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
                 checkSuccesfullyInsertedTable();
@@ -230,4 +232,14 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
             }.getClass().getEnclosingMethod().getName());
         }
     }
+
+    /**
+     * Creates a set of columnFamilies out of a list of hbaseColumns
+     *
+     * @param columns
+     * @return {@link LinkedHashSet}
+     */
+    private static Set<String> getColumnFamilies(List<HBaseColumn> columns) {
+        return columns.stream().map(HBaseColumn::getColumnFamily).distinct().collect(Collectors.toSet());
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5a28d73c/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
index 7a1705f..1a9a424 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
@@ -24,7 +24,6 @@ import java.util.LinkedHashMap;
 import java.util.List;
 
 import org.apache.metamodel.MetaModelException;
-import org.apache.metamodel.schema.Column;
 import org.apache.metamodel.schema.MutableTable;
 
 public class InsertRowTest extends HBaseUpdateCallbackTest {
@@ -135,7 +134,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
             try {
                 final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
-                List<Column> emptyList = new ArrayList<>();
+                List<HBaseColumn> emptyList = new ArrayList<>();
                 new HBaseRowInsertionBuilder(getUpdateCallback(), existingTable, emptyList);
                 fail("Should get an exception that the columns list is empty.");
             } catch (IllegalArgumentException e) {
@@ -258,7 +257,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
                 final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
-                final HBaseColumn[] columns = HBaseColumn.convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
+                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
                 final Object[] values = new String[] { "Values" };
                 new HBaseClient(getDataContext().getConnection()).insertRow(null, columns, values, 0);
                 fail("Should get an exception that tableName is null");
@@ -302,7 +301,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
                 final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
-                final HBaseColumn[] columns = HBaseColumn.convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
+                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
                 new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, null, 0);
                 fail("Should get an exception that values is null");
             } catch (IllegalArgumentException e) {
@@ -325,7 +324,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
                 final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
-                final HBaseColumn[] columns = HBaseColumn.convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
+                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
                 final Object[] values = new String[] { "Values" };
                 new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 10);
                 fail("Should get an exception that the indexOfIdColumn is incorrect");
@@ -349,7 +348,7 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
                 final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
                 final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
                         CF_BAR);
-                final HBaseColumn[] columns = HBaseColumn.convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
+                final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
                 final Object[] values = new String[] { null };
                 new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 0);
                 fail("Should get an exception that the indexOfIdColumn is incorrect");
@@ -437,4 +436,15 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
             }.getClass().getEnclosingMethod().getName());
         }
     }
+
+    /**
+     * Converts a list of {@link HBaseColumn}'s to an array of {@link HBaseColumn}'s
+     *
+     * @param columns
+     * @return Array of {@link HBaseColumn}
+     */
+    private static HBaseColumn[] convertToHBaseColumnsArray(List<HBaseColumn> columns) {
+        return columns.toArray(new HBaseColumn[columns.size()]);
+    }
+
 }


[30/31] metamodel git commit: Made builders package private and added @InterfaceStability.Evolving annotation to HBaseDataContext class.

Posted by ka...@apache.org.
Made builders package private and added @InterfaceStability.Evolving annotation to HBaseDataContext class.


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/c28a8178
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/c28a8178
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/c28a8178

Branch: refs/heads/master
Commit: c28a8178a4f2aa43c2411a943b7acfbd9ce8ee42
Parents: 5e700b5
Author: Arjan Seijkens <a....@quadient.com>
Authored: Tue Jun 19 16:12:07 2018 +0200
Committer: Arjan Seijkens <a....@quadient.com>
Committed: Tue Jun 19 16:12:07 2018 +0200

----------------------------------------------------------------------
 .../java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java   | 2 +-
 .../src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java | 2 ++
 .../java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java   | 2 +-
 .../java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java  | 2 +-
 .../java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java     | 2 +-
 5 files changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/c28a8178/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
index 28963cb..aa722c3 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
@@ -31,7 +31,7 @@ import org.apache.metamodel.util.SimpleTableDef;
 /**
  * A builder-class to create tables in a HBase datastore
  */
-public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseUpdateCallback> {
+class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseUpdateCallback> {
 
     /**
      * Create a {@link HBaseCreateTableBuilder}.

http://git-wip-us.apache.org/repos/asf/metamodel/blob/c28a8178/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
index ec91d8f..48a4695 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
@@ -38,6 +38,7 @@ import org.apache.metamodel.QueryPostprocessDataContext;
 import org.apache.metamodel.UpdateScript;
 import org.apache.metamodel.UpdateSummary;
 import org.apache.metamodel.UpdateableDataContext;
+import org.apache.metamodel.annotations.InterfaceStability;
 import org.apache.metamodel.data.DataSet;
 import org.apache.metamodel.data.DataSetHeader;
 import org.apache.metamodel.data.Row;
@@ -54,6 +55,7 @@ import org.apache.metamodel.util.SimpleTableDef;
 /**
  * MetaModel adaptor for Apache HBase.
  */
+@InterfaceStability.Evolving
 public class HBaseDataContext extends QueryPostprocessDataContext implements UpdateableDataContext {
 
     public static final String FIELD_ID = "_id";

http://git-wip-us.apache.org/repos/asf/metamodel/blob/c28a8178/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
index e8afda8..765ee7a 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
@@ -34,7 +34,7 @@ import org.apache.metamodel.schema.Table;
 /**
  * A builder-class to delete rows based on their keys in a HBase datastore
  */
-public class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
+class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
 
     private final HBaseDataContext _dataContext;
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/c28a8178/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
index fe6e07a..cb06976 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
@@ -38,7 +38,7 @@ import org.apache.metamodel.schema.Column;
 /**
  * A builder-class to insert rows in a HBase datastore.
  */
-public class HBaseRowInsertionBuilder implements RowInsertionBuilder {
+class HBaseRowInsertionBuilder implements RowInsertionBuilder {
     private List<HBaseColumn> _columns = new ArrayList<>();
     private List<Object> _values = new ArrayList<>();
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/c28a8178/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
index 0e1d80c..797a502 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
@@ -26,7 +26,7 @@ import org.apache.metamodel.schema.Table;
 /**
  * A builder-class to drop tables in a HBase datastore
  */
-public class HBaseTableDropBuilder extends AbstractTableDropBuilder {
+class HBaseTableDropBuilder extends AbstractTableDropBuilder {
     private final HBaseUpdateCallback _updateCallback;
 
     public HBaseTableDropBuilder(final Table table, final HBaseUpdateCallback updateCallback) {


[29/31] metamodel git commit: Merge branch 'master' into feature/hbase-improvements

Posted by ka...@apache.org.
Merge branch 'master' into feature/hbase-improvements


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/5e700b5d
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/5e700b5d
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/5e700b5d

Branch: refs/heads/master
Commit: 5e700b5d95184bcceac30cc59707e8510e366c4b
Parents: 9f40295 a868f71
Author: Arjan Seijkens <a....@quadient.com>
Authored: Tue Jun 19 15:43:15 2018 +0200
Committer: Arjan Seijkens <a....@quadient.com>
Committed: Tue Jun 19 15:43:15 2018 +0200

----------------------------------------------------------------------
 .../apache/metamodel/AbstractDataContext.java   |   2 +
 .../org/apache/metamodel/BatchUpdateScript.java |   3 +
 .../java/org/apache/metamodel/DataContext.java  |   2 +
 .../org/apache/metamodel/MetaModelHelper.java   |   2 +-
 .../metamodel/QueryPostprocessDataContext.java  |   2 +
 .../java/org/apache/metamodel/UpdateScript.java |   3 +
 .../apache/metamodel/UpdateableDataContext.java |   3 +
 .../annotations/InterfaceStability.java         |  56 ++++
 .../apache/metamodel/data/AbstractDataSet.java  |   1 -
 .../java/org/apache/metamodel/query/Query.java  |   2 +
 .../org/apache/metamodel/schema/Column.java     |   3 +
 .../org/apache/metamodel/schema/ColumnType.java |   2 +
 .../apache/metamodel/schema/Relationship.java   |   3 +
 .../org/apache/metamodel/schema/Schema.java     |   2 +
 .../java/org/apache/metamodel/schema/Table.java |   3 +
 ...del-integrationtest-configuration.properties |   7 +
 kafka/pom.xml                                   |  52 ++++
 .../kafka/ConsumerAndProducerFactory.java       |  35 +++
 .../kafka/KafkaConsumerAndProducerFactory.java  | 162 +++++++++++
 .../metamodel/kafka/KafkaDataContext.java       | 281 +++++++++++++++++++
 .../apache/metamodel/kafka/KafkaDataSet.java    | 102 +++++++
 .../metamodel/kafka/KafkaInsertBuilder.java     |  54 ++++
 .../metamodel/kafka/KafkaUpdateCallback.java    |  85 ++++++
 .../kafka/KafkaDataContextIntegrationTest.java  | 186 ++++++++++++
 .../metamodel/kafka/KafkaDataContextTest.java   | 172 ++++++++++++
 .../apache/metamodel/kafka/KafkaTestServer.java |  92 ++++++
 .../metamodel/neo4j/ColumnTypeResolver.java     | 137 +++++++++
 .../neo4j/Neo4jCypherQueryBuilder.java          |  53 ++--
 .../metamodel/neo4j/Neo4jDataContext.java       | 278 ++++++++++--------
 .../apache/metamodel/neo4j/Neo4jDataSet.java    |  56 +++-
 .../metamodel/neo4j/Neo4jRequestWrapper.java    |  56 ++--
 .../metamodel/neo4j/ColumnTypeResolverTest.java |  76 +++++
 pom.xml                                         |   1 +
 ...del-integrationtest-configuration.properties |   7 +
 34 files changed, 1785 insertions(+), 196 deletions(-)
----------------------------------------------------------------------



[14/31] metamodel git commit: Merge pull request #3 from GerardDellemann/feature/hbase-improvements-refactoring

Posted by ka...@apache.org.
Merge pull request #3 from GerardDellemann/feature/hbase-improvements-refactoring

Refactoring code and implementing feedback

Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/703fb40e
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/703fb40e
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/703fb40e

Branch: refs/heads/master
Commit: 703fb40e80a91521aca62fae918737aaa5e752bb
Parents: 54a900b 5a28d73
Author: Gerard Dellemann <ge...@gmail.com>
Authored: Thu May 31 13:11:08 2018 +0200
Committer: GitHub <no...@github.com>
Committed: Thu May 31 13:11:08 2018 +0200

----------------------------------------------------------------------
 .../org/apache/metamodel/hbase/HBaseColumn.java | 108 -------------------
 .../hbase/HBaseRowInsertionBuilder.java         |  79 +++++++++++---
 .../org/apache/metamodel/hbase/HBaseTable.java  |  76 +++++++------
 .../metamodel/hbase/HBaseUpdateCallback.java    |  17 +--
 .../apache/metamodel/hbase/CreateTableTest.java |  18 +++-
 .../apache/metamodel/hbase/InsertRowTest.java   |  22 ++--
 6 files changed, 144 insertions(+), 176 deletions(-)
----------------------------------------------------------------------



[22/31] metamodel git commit: - Made HBaseColumn private, even though in the future, I would like to see something like a WideColumn interface to use for Wide Column stores. - Refactored the HBaseCreateTableBuilder so you don't have to pass the column fa

Posted by ka...@apache.org.
- Made HBaseColumn private, even though in the future, I would like to see something like a WideColumn interface to use for Wide Column stores.
- Refactored the HBaseCreateTableBuilder so you don't have to pass the column families to the constructor, but it can determine them based on the columns set on it using the "withColumn" method.
- Refactored the HBaseDeletionBuilder so it uses the "whereItems" instead of the custom "setKey" method to determine what rows it deletes.
- Refactored the HBaseRowInsertionBuilder so it works with "normal" MetaModel columns and it will construct HBaseColumn for internal usage only.
- Removed the no longer needed, not part of the interface createTable method from HBaseUpdateCallback
- Move some test cases to a new HBaseClientTest class, because they specifically test the HBaseClient class
- Updated some test in accordance with the refactorings mentioned above.


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/e37546de
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/e37546de
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/e37546de

Branch: refs/heads/master
Commit: e37546de91200cb13661bf438152e422df6e8814
Parents: 909cb48
Author: Arjan Seijkens <a....@quadient.com>
Authored: Fri Jun 8 14:50:05 2018 +0200
Committer: Arjan Seijkens <a....@quadient.com>
Committed: Fri Jun 8 14:50:05 2018 +0200

----------------------------------------------------------------------
 .../org/apache/metamodel/hbase/HBaseColumn.java |   2 +-
 .../hbase/HBaseCreateTableBuilder.java          |  47 +++++----
 .../metamodel/hbase/HBaseDataContext.java       |   2 +-
 .../hbase/HBaseRowDeletionBuilder.java          |  57 ++++++++--
 .../hbase/HBaseRowInsertionBuilder.java         |  27 +++--
 .../metamodel/hbase/HBaseUpdateCallback.java    |  17 ---
 .../apache/metamodel/hbase/CreateTableTest.java | 104 +++++++++----------
 .../apache/metamodel/hbase/DeleteRowTest.java   |  34 ++----
 .../apache/metamodel/hbase/HBaseClientTest.java |  89 ++++++++++++++++
 .../hbase/HBaseUpdateCallbackTest.java          |   9 +-
 10 files changed, 243 insertions(+), 145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/e37546de/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
index d07c477..f798b07 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
@@ -22,7 +22,7 @@ import org.apache.metamodel.schema.ColumnType;
 import org.apache.metamodel.schema.MutableColumn;
 import org.apache.metamodel.schema.Table;
 
-public final class HBaseColumn extends MutableColumn {
+final class HBaseColumn extends MutableColumn {
     public static final ColumnType DEFAULT_COLUMN_TYPE_FOR_ID_COLUMN = ColumnType.BINARY;
     public static final ColumnType DEFAULT_COLUMN_TYPE_FOR_COLUMN_FAMILIES = ColumnType.LIST;
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/e37546de/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
index 4f8865c..28963cb 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
@@ -19,6 +19,7 @@
 package org.apache.metamodel.hbase;
 
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.create.AbstractTableCreationBuilder;
@@ -32,12 +33,6 @@ import org.apache.metamodel.util.SimpleTableDef;
  */
 public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseUpdateCallback> {
 
-    private Set<String> _columnFamilies;
-
-    public HBaseCreateTableBuilder(final HBaseUpdateCallback updateCallback, final Schema schema, final String name) {
-        this(updateCallback, schema, name, null);
-    }
-
     /**
      * Create a {@link HBaseCreateTableBuilder}.
      * Throws an {@link IllegalArgumentException} if the schema isn't a {@link MutableSchema}.
@@ -46,40 +41,47 @@ public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseU
      * @param name
      * @param columnFamilies
      */
-    public HBaseCreateTableBuilder(final HBaseUpdateCallback updateCallback, final Schema schema, final String name,
-            final Set<String> columnFamilies) {
+    public HBaseCreateTableBuilder(final HBaseUpdateCallback updateCallback, final Schema schema, final String name) {
         super(updateCallback, schema, name);
         if (!(schema instanceof MutableSchema)) {
             throw new IllegalArgumentException("Not a mutable schema: " + schema);
         }
-        this._columnFamilies = columnFamilies;
     }
 
     @Override
     public Table execute() {
-        if (_columnFamilies == null || _columnFamilies.isEmpty()) {
-            throw new MetaModelException("Creating a table without columnFamilies");
+        Set<String> columnFamilies = getColumnFamilies();
+
+        if (columnFamilies == null || columnFamilies.isEmpty()) {
+            throw new MetaModelException("Can't create a table without column families.");
         }
 
         final Table table = getTable();
 
         // Add the table to the datastore
         ((HBaseDataContext) getUpdateCallback().getDataContext()).getHBaseClient().createTable(table.getName(),
-                _columnFamilies);
+                columnFamilies);
 
         // Update the schema
         addNewTableToSchema(table);
         return getSchema().getTableByName(table.getName());
     }
 
-    /**
-     * Set the columnFamilies. This should be used when creating this object using the
-     * {@link HBaseCreateTableBuilder#HBaseCreateTableBuilder(HBaseUpdateCallback, Schema, String)}
-     * constructor
-     * @param columnFamilies
-     */
-    public void setColumnFamilies(final Set<String> columnFamilies) {
-        this._columnFamilies = columnFamilies;
+    private Set<String> getColumnFamilies() {
+        return getTable().getColumns().stream().map(column -> {
+            if (column instanceof HBaseColumn) {
+                return ((HBaseColumn) column).getColumnFamily();
+            } else {
+                String columnName = column.getName();
+
+                String[] columnNameParts = columnName.split(":");
+                if (columnNameParts.length > 0 && columnNameParts.length < 3) {
+                    return columnNameParts[0];
+                } else {
+                    throw new MetaModelException("Can't determine column family for column \"" + columnName + "\".");
+                }
+            }
+        }).distinct().collect(Collectors.toSet());
     }
 
     /**
@@ -90,8 +92,9 @@ public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseU
      */
     private void addNewTableToSchema(final Table table) {
         final MutableSchema schema = (MutableSchema) getSchema();
-        final SimpleTableDef emptyTableDef = new SimpleTableDef(table.getName(), _columnFamilies.toArray(
-                new String[_columnFamilies.size()]));
+        final Set<String> columnFamilies = getColumnFamilies();
+        final SimpleTableDef emptyTableDef = new SimpleTableDef(table.getName(), columnFamilies.toArray(
+                new String[columnFamilies.size()]));
         schema.addTable(new HBaseTable((HBaseDataContext) getUpdateCallback().getDataContext(), emptyTableDef, schema,
                 HBaseConfiguration.DEFAULT_ROW_KEY_TYPE));
     }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/e37546de/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
index 0386ec3..9c989db 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
@@ -251,7 +251,7 @@ public class HBaseDataContext extends QueryPostprocessDataContext implements Upd
         return callback.getUpdateSummary();
     }
 
-    public HBaseClient getHBaseClient() {
+    HBaseClient getHBaseClient() {
         return new HBaseClient(this.getConnection());
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/e37546de/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
index 75301e3..078fe5b 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
@@ -18,8 +18,17 @@
  */
 package org.apache.metamodel.hbase;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.delete.AbstractRowDeletionBuilder;
+import org.apache.metamodel.query.FilterItem;
+import org.apache.metamodel.query.LogicalOperator;
+import org.apache.metamodel.query.OperatorType;
+import org.apache.metamodel.query.SelectItem;
+import org.apache.metamodel.schema.Column;
 import org.apache.metamodel.schema.Table;
 
 /**
@@ -28,7 +37,6 @@ import org.apache.metamodel.schema.Table;
 public class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
 
     private final HBaseDataContext _dataContext;
-    private Object _key;
 
     /**
      * Creates a {@link HBaseRowDeletionBuilder}
@@ -49,13 +57,50 @@ public class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
      */
     @Override
     public synchronized void execute() {
-        if (_key == null) {
-            throw new MetaModelException("Key cannot be null");
+        final List<FilterItem> whereItems = getWhereItems();
+        if (whereItems == null || whereItems.size() == 0) {
+            throw new IllegalArgumentException("HBase currently only supports deleting items by their row key.");
+        }
+
+        final FilterItem filterItem = whereItems.get(0);
+        if (!HBaseDataContext.FIELD_ID.equals(filterItem.getSelectItem().getColumn().getName())) {
+            throw new IllegalArgumentException("HBase currently only supports deleting items by their row key.");
         }
-        _dataContext.getHBaseClient().deleteRow(getTable().getName(), _key);
+
+        getRowKeys(filterItem).forEach(rowKey -> _dataContext.getHBaseClient().deleteRow(getTable().getName(), rowKey));
     }
 
-    public void setKey(final Object key) {
-        this._key = key;
+    private List<Object> getRowKeys(final FilterItem whereItem) {
+        final List<Object> rowKeys = new ArrayList<>();
+
+        if (whereItem.isCompoundFilter()) {
+            final LogicalOperator logicalOperator = whereItem.getLogicalOperator();
+            if (logicalOperator != LogicalOperator.OR) {
+                throw new IllegalStateException(
+                        "HBase currently only supports deleting items by their row key. Violated by operator between where items: "
+                                + whereItem);
+            }
+
+            Arrays.stream(whereItem.getChildItems()).forEach(childItem -> rowKeys.addAll(getRowKeys(childItem)));
+        } else {
+            final OperatorType operator = whereItem.getOperator();
+            if (!OperatorType.EQUALS_TO.equals(operator) && !OperatorType.IN.equals(operator)) {
+                throw new IllegalStateException(
+                        "HBase currently only supports deleting items by their row key. Violated by operator in where item: "
+                                + whereItem);
+            }
+
+            final SelectItem selectItem = whereItem.getSelectItem();
+            final Column column = selectItem.getColumn();
+            final Object operand = whereItem.getOperand();
+
+            if (column == null || operand == null || !column.isPrimaryKey() || selectItem.hasFunction()) {
+                throw new IllegalStateException(
+                        "HBase currently only supports deleting items by their row key. Violated by where item: "
+                                + whereItem);
+            }
+            rowKeys.add(operand);
+        }
+        return rowKeys;
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/e37546de/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
index 0a3a4a3..abbbfee 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
@@ -134,27 +134,42 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
         if (column == null) {
             throw new IllegalArgumentException("Column cannot be null.");
         }
-        if (!(column instanceof HBaseColumn)) {
-            throw new IllegalArgumentException("Column is not an HBaseColumn.");
-        }
+
+        final HBaseColumn hbaseColumn = getHbaseColumn(column);
 
         for (int i = 0; i < columns.size(); i++) {
-            if (columns.get(i).equals(column)) {
+            if (columns.get(i).equals(hbaseColumn)) {
                 values.set(i, value);
                 return this;
             }
         }
 
-        if (column.isPrimaryKey()) {
+        if (hbaseColumn.isPrimaryKey()) {
             _indexOfIdColumn = columns.size();
         }
 
-        columns.add((HBaseColumn) column);
+        columns.add((HBaseColumn) hbaseColumn);
         values.add(value);
 
         return this;
     }
 
+    private HBaseColumn getHbaseColumn(final Column column) {
+        if (column instanceof HBaseColumn) {
+            return (HBaseColumn) column;
+        } else {
+            final String columnName = column.getName();
+            final String[] columnNameParts = columnName.split(":");
+            if (columnNameParts.length == 1) {
+                return new HBaseColumn(columnNameParts[0], getTable());
+            }
+            if (columnNameParts.length == 2) {
+                return new HBaseColumn(columnNameParts[0], columnNameParts[1], getTable());
+            }
+            throw new MetaModelException("Can't determine column family for column \"" + columnName + "\".");
+        }
+    }
+
     @Override
     public RowInsertionBuilder value(final int columnIndex, final Object value) {
         values.set(columnIndex, value);

http://git-wip-us.apache.org/repos/asf/metamodel/blob/e37546de/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
index 5f44200..503de46 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
@@ -18,8 +18,6 @@
  */
 package org.apache.metamodel.hbase;
 
-import java.util.Set;
-
 import org.apache.metamodel.AbstractUpdateCallback;
 import org.apache.metamodel.UpdateCallback;
 import org.apache.metamodel.create.TableCreationBuilder;
@@ -43,18 +41,6 @@ final class HBaseUpdateCallback extends AbstractUpdateCallback implements Update
         return new HBaseCreateTableBuilder(this, schema, name);
     }
 
-    /**
-     * Initiates the building of a table creation operation.
-     * @param schema the schema to create the table in
-     * @param name the name of the new table
-     * @param columnFamilies the columnFamilies of the new table
-     * @return {@link HBaseCreateTableBuilder}
-     */
-    public HBaseCreateTableBuilder createTable(final Schema schema, final String name,
-            final Set<String> columnFamilies) {
-        return new HBaseCreateTableBuilder(this, schema, name, columnFamilies);
-    }
-
     @Override
     public boolean isDropTableSupported() {
         return true;
@@ -65,9 +51,6 @@ final class HBaseUpdateCallback extends AbstractUpdateCallback implements Update
         return new HBaseTableDropBuilder(table, this);
     }
 
-    /**
-     * @throws UnsupportedOperationException use {@link HBaseUpdateCallback#insertInto(String, String)}
-     */
     @Override
     public RowInsertionBuilder insertInto(final Table table) {
         if (table instanceof HBaseTable) {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/e37546de/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
index 0415c12..4c71a47 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
@@ -18,20 +18,18 @@
  */
 package org.apache.metamodel.hbase;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.LinkedHashSet;
-import java.util.Set;
 
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.schema.ImmutableSchema;
+import org.apache.metamodel.schema.Table;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
-import com.google.common.collect.Sets;
-
 public class CreateTableTest extends HBaseUpdateCallbackTest {
     @Rule
     public ExpectedException exception = ExpectedException.none();
@@ -58,73 +56,36 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
     }
 
     /**
-     * Create a table without columnFamilies, should throw a MetaModelException
+     * Create a table without column families, should throw a MetaModelException
      */
     @Test
-    public void testCreateTableWithoutColumnFamilies() {
+    public void testWithoutColumnFamilies() {
         exception.expect(MetaModelException.class);
-        exception.expectMessage("Creating a table without columnFamilies");
+        exception.expectMessage("Can't create a table without column families.");
 
         getUpdateCallback().createTable(getSchema(), TABLE_NAME).execute();
     }
 
     /**
-     * Create a table with columnFamilies null, should throw a MetaModelException
-     */
-    @Test
-    public void testColumnFamiliesNull() {
-        exception.expect(MetaModelException.class);
-        exception.expectMessage("Creating a table without columnFamilies");
-
-        getUpdateCallback().createTable(getSchema(), TABLE_NAME, null).execute();
-    }
-
-    /**
-     * Create a table with columnFamilies empty, should throw a MetaModelException
-     */
-    @Test
-    public void testColumnFamiliesEmpty() {
-        exception.expect(MetaModelException.class);
-        exception.expectMessage("Creating a table without columnFamilies");
-
-        final Set<String> columnFamilies = new LinkedHashSet<String>();
-        getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
-    }
-
-    /**
-     * Creating a HBaseClient with the tableName null, should throw a exception
-     */
-    @Test
-    public void testCreatingTheHBaseClientWithTableNameNull() {
-        exception.expect(IllegalArgumentException.class);
-        exception.expectMessage("Can't create a table without having the tableName or columnFamilies");
-
-        final Set<String> columnFamilies = new LinkedHashSet<>();
-        columnFamilies.add("1");
-        new HBaseClient(getDataContext().getConnection()).createTable(null, columnFamilies);
-    }
-
-    /**
-     * Creating a HBaseClient with the tableName null, should throw a exception
+     * Create a table without column families, should throw a MetaModelException
      */
     @Test
-    public void testCreatingTheHBaseClientWithColumnFamiliesNull() {
+    public void testWithEmptyColumn() {
         exception.expect(IllegalArgumentException.class);
-        exception.expectMessage("Can't create a table without having the tableName or columnFamilies");
+        exception.expectMessage("Family name can not be empty");
 
-        new HBaseClient(getDataContext().getConnection()).createTable("1", null);
+        getUpdateCallback().createTable(getSchema(), TABLE_NAME).withColumn("").execute();
     }
 
     /**
-     * Creating a HBaseClient with the tableName null, should throw a exception
+     * Create a table without column families, should throw a MetaModelException
      */
     @Test
-    public void testCreatingTheHBaseClientWithColumnFamiliesEmpty() {
-        exception.expect(IllegalArgumentException.class);
-        exception.expectMessage("Can't create a table without having the tableName or columnFamilies");
+    public void testWithIndeterminableColumn() {
+        exception.expect(MetaModelException.class);
+        exception.expectMessage("Can't determine column family for column \"a:b:c\".");
 
-        final Set<String> columnFamilies = new LinkedHashSet<>();
-        new HBaseClient(getDataContext().getConnection()).createTable("1", columnFamilies);
+        getUpdateCallback().createTable(getSchema(), TABLE_NAME).withColumn("a:b:c").execute();
     }
 
     /**
@@ -137,9 +98,17 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
         final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
                 .createTable(getSchema(), TABLE_NAME);
 
-        hBaseCreateTableBuilder.setColumnFamilies(Sets.newHashSet(CF_FOO, CF_BAR));
+        hBaseCreateTableBuilder.withColumn(CF_FOO);
+        hBaseCreateTableBuilder.withColumn(CF_BAR);
         hBaseCreateTableBuilder.execute();
         checkSuccesfullyInsertedTable();
+
+        final Table table = getDataContext().getDefaultSchema().getTableByName(TABLE_NAME);
+        assertTrue(table instanceof HBaseTable);
+
+        // Assert that the Table has 3 column families, a default "_id" one, and two based on the column families for
+        // the columns.
+        assertEquals(3, ((HBaseTable) table).getColumnFamilies().size());
     }
 
     /**
@@ -148,13 +117,23 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      * @throws IOException
      */
     @Test
-    public void testSettingColumnFamiliesAfterConstrutor() throws IOException {
+    public void testCreateTableWithQualifiedColumns() throws IOException {
         final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
                 .createTable(getSchema(), TABLE_NAME);
 
-        hBaseCreateTableBuilder.setColumnFamilies(Sets.newHashSet(CF_FOO, CF_BAR));
+        hBaseCreateTableBuilder.withColumn(CF_FOO + ":" + Q_BAH);
+        hBaseCreateTableBuilder.withColumn(CF_FOO + ":" + Q_HELLO);
+        hBaseCreateTableBuilder.withColumn(CF_BAR + ":" + Q_HEY);
+        hBaseCreateTableBuilder.withColumn(CF_BAR + ":" + Q_HI);
         hBaseCreateTableBuilder.execute();
         checkSuccesfullyInsertedTable();
+
+        final Table table = getDataContext().getDefaultSchema().getTableByName(TABLE_NAME);
+        assertTrue(table instanceof HBaseTable);
+
+        // Assert that the Table has 3 column families, a default "_id" one, and two based on the column families for
+        // the columns.
+        assertEquals(3, ((HBaseTable) table).getColumnFamilies().size());
     }
 
     /**
@@ -164,7 +143,18 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreateTableColumnFamiliesInConstrutor() throws IOException {
-        getUpdateCallback().createTable(getSchema(), TABLE_NAME, Sets.newHashSet(CF_FOO, CF_BAR)).execute();
+        final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
+                .createTable(getSchema(), TABLE_NAME);
+
+        hBaseCreateTableBuilder.withColumn(HBaseDataContext.FIELD_ID);
+        hBaseCreateTableBuilder.withColumn(CF_BAR);
+        hBaseCreateTableBuilder.execute();
         checkSuccesfullyInsertedTable();
+
+        final Table table = getDataContext().getDefaultSchema().getTableByName(TABLE_NAME);
+        assertTrue(table instanceof HBaseTable);
+
+        // Assert that the Table has 2 column families.
+        assertEquals(2, ((HBaseTable) table).getColumnFamilies().size());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/metamodel/blob/e37546de/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
index 5a95032..2eefb06 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.Map;
 
-import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.insert.RowInsertionBuilder;
 import org.apache.metamodel.schema.MutableTable;
 import org.junit.Rule;
@@ -76,35 +75,14 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testNotSettingRowkey() throws IOException {
-        exception.expect(MetaModelException.class);
-        exception.expectMessage("Key cannot be null");
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("HBase currently only supports deleting items by their row key.");
 
         final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                 CF_BAR);
         getUpdateCallback().deleteFrom(existingTable).execute();
     }
 
-    /**
-     * Creating a HBaseClient with the tableName null, should throw a exception
-     */
-    @Test
-    public void testCreatingTheHBaseClientWithTableNameNull() {
-        exception.expect(IllegalArgumentException.class);
-        exception.expectMessage("Can't delete a row without having tableName or rowKey");
-
-        new HBaseClient(getDataContext().getConnection()).deleteRow(null, new String("1"));
-    }
-
-    /**
-     * Creating a HBaseClient with the rowKey null, should throw a exception
-     */
-    @Test
-    public void testCreatingTheHBaseClientWithRowKeyNull() {
-        exception.expect(IllegalArgumentException.class);
-        exception.expectMessage("Can't delete a row without having tableName or rowKey");
-
-        new HBaseClient(getDataContext().getConnection()).deleteRow("tableName", null);
-    }
 
     /**
      * Goodflow. Deleting a row, that doesn't exist, should not throw an exception
@@ -119,7 +97,7 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
         checkRows(false, false);
         final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
                 existingTable);
-        rowDeletionBuilder.setKey(RK_1);
+        rowDeletionBuilder.where(HBaseDataContext.FIELD_ID).eq(RK_1);
         rowDeletionBuilder.execute();
         checkRows(false, false);
     }
@@ -137,13 +115,13 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
         checkRows(false, false);
         final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
                 existingTable);
-        rowDeletionBuilder.setKey("");
+        rowDeletionBuilder.where(HBaseDataContext.FIELD_ID).eq("");
         rowDeletionBuilder.execute();
         checkRows(false, false);
     }
 
     /**
-     * Goodflow. Deleting a row succesfully.
+     * Goodflow. Deleting a row successfully.
      *
      * @throws IOException
      */
@@ -160,7 +138,7 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
         checkRows(true, false);
         final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
                 existingTable);
-        rowDeletionBuilder.setKey(RK_1);
+        rowDeletionBuilder.where(HBaseDataContext.FIELD_ID).eq(RK_1);
         rowDeletionBuilder.execute();
         checkRows(false, false);
     }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/e37546de/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java
new file mode 100644
index 0000000..35aa310
--- /dev/null
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java
@@ -0,0 +1,89 @@
+/**
+ * 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.metamodel.hbase;
+
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class HBaseClientTest extends HBaseTestCase {
+    @Rule
+    public ExpectedException exception = ExpectedException.none();
+
+    /**
+     * Creating a HBaseClient with the tableName null, should throw a exception
+     */
+    @Test
+    public void testCreatingTheHBaseClientWithTableNameNull() {
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Can't create a table without having the tableName or columnFamilies");
+
+        final Set<String> columnFamilies = new LinkedHashSet<>();
+        columnFamilies.add("1");
+        new HBaseClient(getDataContext().getConnection()).createTable(null, columnFamilies);
+    }
+
+    /**
+     * Creating a HBaseClient with the tableName null, should throw a exception
+     */
+    @Test
+    public void testCreatingTheHBaseClientWithColumnFamiliesNull() {
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Can't create a table without having the tableName or columnFamilies");
+
+        new HBaseClient(getDataContext().getConnection()).createTable("1", null);
+    }
+
+    /**
+     * Creating a HBaseClient with the tableName null, should throw a exception
+     */
+    @Test
+    public void testCreatingTheHBaseClientWithColumnFamiliesEmpty() {
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Can't create a table without having the tableName or columnFamilies");
+
+        final Set<String> columnFamilies = new LinkedHashSet<>();
+        new HBaseClient(getDataContext().getConnection()).createTable("1", columnFamilies);
+    }
+
+    /**
+     * Creating a HBaseClient with the tableName null, should throw a exception
+     */
+    @Test
+    public void testDeleteRowWithoutTableName() {
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Can't delete a row without having tableName or rowKey");
+
+        new HBaseClient(getDataContext().getConnection()).deleteRow(null, new String("1"));
+    }
+
+    /**
+     * Creating a HBaseClient with the rowKey null, should throw a exception
+     */
+    @Test
+    public void testCreatingTheHBaseClientWithRowKeyNull() {
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Can't delete a row without having tableName or rowKey");
+
+        new HBaseClient(getDataContext().getConnection()).deleteRow("tableName", null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/metamodel/blob/e37546de/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
index 14205fb..7d03641 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
@@ -24,10 +24,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.stream.Collectors;
 
 import org.apache.hadoop.hbase.TableName;
@@ -108,11 +106,8 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
      */
     protected HBaseTable createAndAddTableToDatastore(final String tableName, final String idColumn,
             final String columnFamily1, final String columnFamily2) throws IOException {
-        final Set<String> columnFamilies = new LinkedHashSet<>();
-        columnFamilies.add(idColumn);
-        columnFamilies.add(columnFamily1);
-        columnFamilies.add(columnFamily2);
-        updateCallback.createTable(schema, tableName, columnFamilies).execute();
+        updateCallback.createTable(schema, tableName).withColumn(columnFamily1).withColumn(columnFamily2).execute();
+
         checkSuccesfullyInsertedTable();
         return (HBaseTable) getDataContext().getDefaultSchema().getTableByName(tableName);
     }


[10/31] metamodel git commit: HBase Improvements - did some refactoring, added comments, added unittest

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
new file mode 100644
index 0000000..622b70b
--- /dev/null
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
@@ -0,0 +1,435 @@
+/**
+ * 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.metamodel.hbase;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+
+import org.apache.metamodel.MetaModelException;
+import org.apache.metamodel.schema.Column;
+import org.apache.metamodel.schema.MutableTable;
+
+public class InsertRowTest extends HBaseUpdateCallbackTest {
+
+    /**
+     * Check if inserting into a table is supported
+     * @throws IOException
+     */
+    public void testInsertSupported() throws IOException {
+        assertTrue(getUpdateCallback().isInsertSupported());
+    }
+
+    /**
+     * Using only the table parameter, should throw an exception
+     * @throws IOException
+     */
+    public void testOnlyUsingTableParameter() throws IOException {
+        if (isConfigured()) {
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                getUpdateCallback().insertInto(existingTable);
+                fail("Should get an exception that this method is not supported");
+            } catch (UnsupportedOperationException e) {
+                assertEquals("We need an explicit list of columns when inserting into an HBase table.", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Having the table type wrong, should throw an exception
+     * @throws IOException
+     */
+    public void testWrongTableType() throws IOException {
+        if (isConfigured()) {
+            MutableTable mutableTable = new MutableTable();
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
+                        CF_FOO, CF_BAR);
+                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                getUpdateCallback().insertInto(mutableTable, columns);
+                fail("Should get an exception that the type of the table is wrong.");
+            } catch (IllegalArgumentException e) {
+                assertEquals("Not an HBase table: " + mutableTable, e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Having the columns parameter null at the updateCallBack, should throw an exception
+     * @throws IOException
+     */
+    public void testColumnsNullAtUpdateCallBack() throws IOException {
+        if (isConfigured()) {
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                getUpdateCallback().insertInto(existingTable, null);
+                fail("Should get an exception that the columns list is null.");
+            } catch (IllegalArgumentException e) {
+                assertEquals("The hbaseColumns list is null or empty", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Having the columns parameter empty at the updateCallBack, should throw an exception
+     * @throws IOException
+     */
+    public void testColumnsEmptyAtUpdateCallBack() throws IOException {
+        if (isConfigured()) {
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                getUpdateCallback().insertInto(existingTable, new ArrayList<HBaseColumn>());
+                fail("Should get an exception that the columns list is empty.");
+            } catch (IllegalArgumentException e) {
+                assertEquals("The hbaseColumns list is null or empty", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Having the columns parameter empty at the builder, should throw an exception
+     * @throws IOException
+     */
+    public void testColumnsEmptyAtBuilder() throws IOException {
+        if (isConfigured()) {
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                List<Column> emptyList = new ArrayList<>();
+                new HBaseRowInsertionBuilder(getUpdateCallback(), existingTable, emptyList);
+                fail("Should get an exception that the columns list is empty.");
+            } catch (IllegalArgumentException e) {
+                assertEquals("The hbaseColumns list is null or empty", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Using a table that doesn't exist in the schema, should throw an exception
+     * @throws IOException
+     */
+    public void testTableThatDoesntExist() throws IOException {
+        if (isConfigured()) {
+            final HBaseTable wrongTable = createHBaseTable("NewTableNotInSchema", HBaseDataContext.FIELD_ID, "cf1",
+                    "cf2", null);
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
+                        CF_FOO, CF_BAR);
+                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                getUpdateCallback().insertInto(wrongTable, columns);
+                fail("Should get an exception that the table isn't in the schema.");
+            } catch (MetaModelException e) {
+                assertEquals("Trying to insert data into table: " + wrongTable.getName() + ", which doesn't exist yet",
+                        e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * If the ID-column doesn't exist in the columns array, then a exception should be thrown
+     * @throws IOException
+     */
+    public void testIDColumnDoesntExistInColumnsArray() throws IOException {
+        if (isConfigured()) {
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, null, CF_FOO, CF_BAR);
+                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                getUpdateCallback().insertInto(existingTable, columns);
+                fail("Should get an exception that ID-column doesn't exist.");
+            } catch (MetaModelException e) {
+                assertEquals("The ID-Column was not found", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * If the column family doesn't exist in the table (wrong columnFamily), then a exception should be thrown
+     * @throws IOException
+     */
+    public void testColumnFamilyDoesntExistsBecauseItsNull() throws IOException {
+        if (isConfigured()) {
+            final String wrongColumnFamily = "wrongColumnFamily";
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
+                        CF_FOO, CF_BAR);
+                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        wrongColumnFamily, null);
+                getUpdateCallback().insertInto(wrongTable, columns);
+                fail("Should get an exception that the columnFamily doesn't exist.");
+            } catch (MetaModelException e) {
+                assertEquals(String.format("ColumnFamily: %s doesn't exist in the schema of the table",
+                        wrongColumnFamily), e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * If the column family doesn't exist in the table (new columnFamily), then a exception should be thrown
+     * @throws IOException
+     */
+    public void testColumnFamilyDoesntExistsBecauseItsNew() throws IOException {
+        if (isConfigured()) {
+            final String wrongColumnFamily = "newColumnFamily";
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
+                        CF_FOO, CF_BAR);
+                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
+                        wrongColumnFamily);
+                getUpdateCallback().insertInto(wrongTable, columns);
+                fail("Should get an exception that the columnFamily doesn't exist.");
+            } catch (MetaModelException e) {
+                assertEquals(String.format("ColumnFamily: %s doesn't exist in the schema of the table",
+                        wrongColumnFamily), e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Creating a HBaseClient with the tableName null, should throw a exception
+     */
+    public void testCreatingTheHBaseClientWithTableNameNull() {
+        if (isConfigured()) {
+            try {
+                final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+                final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                final HBaseColumn[] columns = HBaseColumn.convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
+                final Object[] values = new String[] { "Values" };
+                new HBaseClient(getDataContext().getConnection()).insertRow(null, columns, values, 0);
+                fail("Should get an exception that tableName is null");
+            } catch (IllegalArgumentException e) {
+                assertEquals(
+                        "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
+                                .getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Creating a HBaseClient with the columns null, should throw a exception
+     */
+    public void testCreatingTheHBaseClientWithColumnsNull() {
+        if (isConfigured()) {
+            try {
+                final Object[] values = new String[] { "Values" };
+                new HBaseClient(getDataContext().getConnection()).insertRow("tableName", null, values, 0);
+                fail("Should get an exception that columns is null");
+            } catch (IllegalArgumentException e) {
+                assertEquals(
+                        "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
+                                .getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Creating a HBaseClient with the values null, should throw a exception
+     */
+    public void testCreatingTheHBaseClientWithValuesNull() {
+        if (isConfigured()) {
+            try {
+                final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+                final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                final HBaseColumn[] columns = HBaseColumn.convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
+                new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, null, 0);
+                fail("Should get an exception that values is null");
+            } catch (IllegalArgumentException e) {
+                assertEquals(
+                        "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
+                                .getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Creating a HBaseClient with the indexOfIdColumn out of bounce, should throw a exception
+     */
+    public void testCreatingTheHBaseClientWithIndexOfIdColumnOutOfBounce() {
+        if (isConfigured()) {
+            try {
+                final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+                final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                final HBaseColumn[] columns = HBaseColumn.convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
+                final Object[] values = new String[] { "Values" };
+                new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 10);
+                fail("Should get an exception that the indexOfIdColumn is incorrect");
+            } catch (IllegalArgumentException e) {
+                assertEquals(
+                        "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
+                                .getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Creating a HBaseClient with the rowKey null, should throw a exception
+     */
+    public void testCreatingTheHBaseClientWithRowKeyNull() {
+        if (isConfigured()) {
+            try {
+                final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+                final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                final HBaseColumn[] columns = HBaseColumn.convertToHBaseColumnsArray(getHBaseColumnsFromMap(row));
+                final Object[] values = new String[] { null };
+                new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 0);
+                fail("Should get an exception that the indexOfIdColumn is incorrect");
+            } catch (IllegalArgumentException e) {
+                assertEquals(
+                        "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
+                                .getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Goodflow. Using an existing table and columns, should work
+     */
+    public void testInsertIntoWithoutExecute() {
+        if (isConfigured()) {
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
+                        CF_FOO, CF_BAR);
+                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+                getUpdateCallback().insertInto(existingTable, columns);
+            } catch (Exception e) {
+                fail("No exception should be thrown, when inserting into an existing table.");
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    // /**
+    // * Goodflow. Executing the insertInto with the Values being null, should not throw an exception
+    // */
+    // public void testNotSettingTheValues() {
+    // if (isConfigured()) {
+    // try {
+    // final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+    // CF_BAR);
+    // final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
+    // CF_FOO, CF_BAR);
+    // final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+    //
+    // checkRows(false);
+    // final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable,
+    // columns);
+    // rowInsertionBuilder.execute();
+    // checkRows(false);
+    // } catch (Exception e) {
+    // fail("No exception should be thrown, when inserting without values.");
+    // }
+    // } else {
+    // warnAboutANotExecutedTest(getClass().getName(), new Object() {
+    // }.getClass().getEnclosingMethod().getName());
+    // }
+    // }
+
+    /**
+     * Goodflow. Inserting a row succesfully (with values set)
+     */
+    public void testInsertingSuccesfully() {
+        if (isConfigured()) {
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
+                        CF_FOO, CF_BAR);
+                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+
+                checkRows(false);
+                final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable,
+                        columns);
+                setValuesInInsertionBuilder(row, rowInsertionBuilder);
+                rowInsertionBuilder.execute();
+                checkRows(true);
+            } catch (Exception e) {
+                fail("No exception should be thrown, when inserting with values.");
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+}


[20/31] metamodel git commit: HBaseClient.java: - Remove overabundant `; ` on line 111 ("try (final Table table = _connection.getTable(TableName.valueOf(tableName)); ) {"). - Remove overabundant " == true" on line 112 ("if (rowExists(table, rowKeyAsByteArr

Posted by ka...@apache.org.
HBaseClient.java:
- Remove overabundant `;` on line 111 ("try (final Table table = _connection.getTable(TableName.valueOf(tableName));) {").
- Remove overabundant " == true" on line 112 ("if (rowExists(table, rowKeyAsByteArray) == true) {").
- On line 115 use a Marker object when inserting variables into log messages.
- Add final modifier on line 144 ("public void createTable(String tableName, Set<String> columnFamilies) {").

HBaseCreateTableBuilder.java:
- On line 81 make the parameters of the setColumnFamilies method final.
HBaseRow.java:
- Revert remmoval of empty line (62).
HBaseRowDeletionBuilder.java:
- The _dataContext field member has to be made final
- On line 58 make the key par


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/5313a492
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/5313a492
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/5313a492

Branch: refs/heads/master
Commit: 5313a492f7c733be367c2738af14bc566a6090e1
Parents: 4a928d1
Author: Arjan Seijkens <a....@quadient.com>
Authored: Wed Jun 6 10:12:02 2018 +0200
Committer: Arjan Seijkens <a....@quadient.com>
Committed: Wed Jun 6 10:12:02 2018 +0200

----------------------------------------------------------------------
 .../org/apache/metamodel/hbase/HBaseClient.java |   8 +-
 .../hbase/HBaseCreateTableBuilder.java          |  12 +-
 .../org/apache/metamodel/hbase/HBaseRow.java    |   1 +
 .../hbase/HBaseRowDeletionBuilder.java          |   6 +-
 .../apache/metamodel/hbase/CreateTableTest.java | 159 +++-----
 .../apache/metamodel/hbase/DeleteRowTest.java   | 155 ++++----
 .../apache/metamodel/hbase/DropTableTest.java   |  47 ++-
 .../apache/metamodel/hbase/HBaseTestCase.java   |   3 +-
 .../hbase/HBaseUpdateCallbackTest.java          |  88 ++---
 .../apache/metamodel/hbase/InsertRowTest.java   | 363 +++++++++----------
 10 files changed, 365 insertions(+), 477 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/5313a492/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
index a9040fc..a868a93 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
@@ -108,11 +108,11 @@ final class HBaseClient {
         }
         byte[] rowKeyAsByteArray = getValueAsByteArray(rowKey);
         if (rowKeyAsByteArray.length > 0) {
-            try (final Table table = _connection.getTable(TableName.valueOf(tableName));) {
-                if (rowExists(table, rowKeyAsByteArray) == true) {
+            try (final Table table = _connection.getTable(TableName.valueOf(tableName))) {
+                if (rowExists(table, rowKeyAsByteArray)) {
                     table.delete(new Delete(rowKeyAsByteArray));
                 } else {
-                    logger.warn("Rowkey with value " + rowKey.toString() + " doesn't exist in the table");
+                    logger.warn("Rowkey with value {} doesn't exist in the table", rowKey.toString());
                 }
             } catch (IOException e) {
                 throw new MetaModelException(e);
@@ -141,7 +141,7 @@ final class HBaseClient {
      * @throws IllegalArgumentException when any parameter is null
      * @throws MetaModelException when a {@link IOException} is caught
      */
-    public void createTable(String tableName, Set<String> columnFamilies) {
+    public void createTable(final String tableName, final Set<String> columnFamilies) {
         if (tableName == null || columnFamilies == null || columnFamilies.isEmpty()) {
             throw new IllegalArgumentException("Can't create a table without having the tableName or columnFamilies");
         }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5313a492/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
index a7c760b..4f8865c 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
@@ -34,7 +34,7 @@ public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseU
 
     private Set<String> _columnFamilies;
 
-    public HBaseCreateTableBuilder(HBaseUpdateCallback updateCallback, Schema schema, String name) {
+    public HBaseCreateTableBuilder(final HBaseUpdateCallback updateCallback, final Schema schema, final String name) {
         this(updateCallback, schema, name, null);
     }
 
@@ -46,8 +46,8 @@ public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseU
      * @param name
      * @param columnFamilies
      */
-    public HBaseCreateTableBuilder(HBaseUpdateCallback updateCallback, Schema schema, String name,
-            Set<String> columnFamilies) {
+    public HBaseCreateTableBuilder(final HBaseUpdateCallback updateCallback, final Schema schema, final String name,
+            final Set<String> columnFamilies) {
         super(updateCallback, schema, name);
         if (!(schema instanceof MutableSchema)) {
             throw new IllegalArgumentException("Not a mutable schema: " + schema);
@@ -73,12 +73,12 @@ public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseU
     }
 
     /**
-     * Set the columnFamilies. This should be used when creating this object using the 
-     * {@link HBaseCreateTableBuilder#HBaseCreateTableBuilder(HBaseUpdateCallback, Schema, String)} 
+     * Set the columnFamilies. This should be used when creating this object using the
+     * {@link HBaseCreateTableBuilder#HBaseCreateTableBuilder(HBaseUpdateCallback, Schema, String)}
      * constructor
      * @param columnFamilies
      */
-    public void setColumnFamilies(Set<String> columnFamilies) {
+    public void setColumnFamilies(final Set<String> columnFamilies) {
         this._columnFamilies = columnFamilies;
     }
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5313a492/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java
index 79267da..b091ae1 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRow.java
@@ -59,6 +59,7 @@ final class HBaseRow extends AbstractRow implements Row {
             }
             return rowKey;
         }
+
         final int colonIndex = name.indexOf(':');
         if (colonIndex != -1) {
             byte[] family = name.substring(0, colonIndex).getBytes();

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5313a492/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
index 9b7f9ab..75301e3 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
@@ -27,7 +27,7 @@ import org.apache.metamodel.schema.Table;
  */
 public class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
 
-    private HBaseDataContext _dataContext;
+    private final HBaseDataContext _dataContext;
     private Object _key;
 
     /**
@@ -44,7 +44,7 @@ public class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
         this._dataContext = dataContext;
     }
 
-    /** 
+    /**
      * @throws MetaModelException when value is null
      */
     @Override
@@ -55,7 +55,7 @@ public class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
         _dataContext.getHBaseClient().deleteRow(getTable().getName(), _key);
     }
 
-    public void setKey(Object key) {
+    public void setKey(final Object key) {
         this._key = key;
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5313a492/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
index 289212b..0415c12 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
@@ -18,20 +18,23 @@
  */
 package org.apache.metamodel.hbase;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
-import java.util.List;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.schema.ImmutableSchema;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import com.google.common.collect.Sets;
 
 public class CreateTableTest extends HBaseUpdateCallbackTest {
+    @Rule
+    public ExpectedException exception = ExpectedException.none();
 
     /**
      * Check if creating table is supported
@@ -47,12 +50,11 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
     @Test
     public void testWrongSchema() {
         final ImmutableSchema immutableSchema = new ImmutableSchema(getSchema());
-        try {
-            getUpdateCallback().createTable(immutableSchema, TABLE_NAME).execute();
-            fail("Should get an exception that the schema isn't mutable");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Not a mutable schema: " + immutableSchema, e.getMessage());
-        }
+
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Not a mutable schema: " + immutableSchema);
+
+        getUpdateCallback().createTable(immutableSchema, TABLE_NAME).execute();
     }
 
     /**
@@ -60,12 +62,10 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreateTableWithoutColumnFamilies() {
-        try {
-            getUpdateCallback().createTable(getSchema(), TABLE_NAME).execute();
-            fail("Should get an exception that the columnFamilies haven't been set");
-        } catch (MetaModelException e) {
-            assertEquals("Creating a table without columnFamilies", e.getMessage());
-        }
+        exception.expect(MetaModelException.class);
+        exception.expectMessage("Creating a table without columnFamilies");
+
+        getUpdateCallback().createTable(getSchema(), TABLE_NAME).execute();
     }
 
     /**
@@ -73,12 +73,10 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testColumnFamiliesNull() {
-        try {
-            getUpdateCallback().createTable(getSchema(), TABLE_NAME, null).execute();
-            fail("Should get an exception that the columnFamilies haven't been set");
-        } catch (MetaModelException e) {
-            assertEquals("Creating a table without columnFamilies", e.getMessage());
-        }
+        exception.expect(MetaModelException.class);
+        exception.expectMessage("Creating a table without columnFamilies");
+
+        getUpdateCallback().createTable(getSchema(), TABLE_NAME, null).execute();
     }
 
     /**
@@ -86,13 +84,11 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testColumnFamiliesEmpty() {
-        try {
-            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<String>();
-            getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
-            fail("Should get an exception that the columnFamilies haven't been set");
-        } catch (MetaModelException e) {
-            assertEquals("Creating a table without columnFamilies", e.getMessage());
-        }
+        exception.expect(MetaModelException.class);
+        exception.expectMessage("Creating a table without columnFamilies");
+
+        final Set<String> columnFamilies = new LinkedHashSet<String>();
+        getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
     }
 
     /**
@@ -100,14 +96,12 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreatingTheHBaseClientWithTableNameNull() {
-        try {
-            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
-            columnFamilies.add("1");
-            new HBaseClient(getDataContext().getConnection()).createTable(null, columnFamilies);
-            fail("Should get an exception that tableName is null");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Can't create a table without having the tableName or columnFamilies");
+
+        final Set<String> columnFamilies = new LinkedHashSet<>();
+        columnFamilies.add("1");
+        new HBaseClient(getDataContext().getConnection()).createTable(null, columnFamilies);
     }
 
     /**
@@ -115,12 +109,10 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreatingTheHBaseClientWithColumnFamiliesNull() {
-        try {
-            new HBaseClient(getDataContext().getConnection()).createTable("1", null);
-            fail("Should get an exception that columnFamilies is null");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Can't create a table without having the tableName or columnFamilies");
+
+        new HBaseClient(getDataContext().getConnection()).createTable("1", null);
     }
 
     /**
@@ -128,13 +120,11 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreatingTheHBaseClientWithColumnFamiliesEmpty() {
-        try {
-            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
-            new HBaseClient(getDataContext().getConnection()).createTable("1", columnFamilies);
-            fail("Should get an exception that columnFamilies is empty");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Can't create a table without having the tableName or columnFamilies");
+
+        final Set<String> columnFamilies = new LinkedHashSet<>();
+        new HBaseClient(getDataContext().getConnection()).createTable("1", columnFamilies);
     }
 
     /**
@@ -144,66 +134,37 @@ public class CreateTableTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreateTableWithoutIDColumn() throws IOException {
-        final HBaseTable table = createHBaseTable(TABLE_NAME, null, CF_FOO, CF_BAR, null);
-        final LinkedHashMap<HBaseColumn, Object> row = createRow(table, null, CF_FOO, CF_BAR, false);
-        final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
-        try {
-            final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
-                    .createTable(getSchema(), TABLE_NAME);
-
-            hBaseCreateTableBuilder.setColumnFamilies(columnFamilies);
-            hBaseCreateTableBuilder.execute();
-            checkSuccesfullyInsertedTable();
-        } catch (Exception e) {
-            fail("Should not get an exception (that the ID-column is missing)");
-        }
+        final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
+                .createTable(getSchema(), TABLE_NAME);
+
+        hBaseCreateTableBuilder.setColumnFamilies(Sets.newHashSet(CF_FOO, CF_BAR));
+        hBaseCreateTableBuilder.execute();
+        checkSuccesfullyInsertedTable();
     }
 
     /**
      * Goodflow. Create a table including the ID-Column (columnFamilies not in constructor), should work
+     *
+     * @throws IOException
      */
     @Test
-    public void testSettingColumnFamiliesAfterConstrutor() {
-        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-        final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
-                false);
-        final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
-        try {
-            final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
-                    .createTable(getSchema(), TABLE_NAME);
-
-            hBaseCreateTableBuilder.setColumnFamilies(columnFamilies);
-            hBaseCreateTableBuilder.execute();
-            checkSuccesfullyInsertedTable();
-        } catch (Exception e) {
-            fail("Should not get an exception");
-        }
-    }
+    public void testSettingColumnFamiliesAfterConstrutor() throws IOException {
+        final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
+                .createTable(getSchema(), TABLE_NAME);
 
-    /**
-     * Goodflow. Create a table including the ID-Column (columnFamilies in constructor), should work
-     */
-    @Test
-    public void testCreateTableColumnFamiliesInConstrutor() {
-        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-        final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
-                false);
-        final Set<String> columnFamilies = getColumnFamilies(getHBaseColumnsFromRow(row));
-        try {
-            getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
-            checkSuccesfullyInsertedTable();
-        } catch (Exception e) {
-            fail("Should not get an exception");
-        }
+        hBaseCreateTableBuilder.setColumnFamilies(Sets.newHashSet(CF_FOO, CF_BAR));
+        hBaseCreateTableBuilder.execute();
+        checkSuccesfullyInsertedTable();
     }
 
     /**
-     * Creates a set of columnFamilies out of a list of hbaseColumns
+     * Goodflow. Create a table including the ID-Column (columnFamilies in constructor), should work
      *
-     * @param columns
-     * @return {@link Set}<{@link String}> of columnFamilies
+     * @throws IOException
      */
-    private static Set<String> getColumnFamilies(List<HBaseColumn> columns) {
-        return columns.stream().map(HBaseColumn::getColumnFamily).distinct().collect(Collectors.toSet());
+    @Test
+    public void testCreateTableColumnFamiliesInConstrutor() throws IOException {
+        getUpdateCallback().createTable(getSchema(), TABLE_NAME, Sets.newHashSet(CF_FOO, CF_BAR)).execute();
+        checkSuccesfullyInsertedTable();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5313a492/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
index 7b7eeab..7aeb938 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
@@ -21,14 +21,18 @@ package org.apache.metamodel.hbase;
 import static org.junit.Assert.*;
 
 import java.io.IOException;
-import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.schema.MutableTable;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 
 public class DeleteRowTest extends HBaseUpdateCallbackTest {
+    @Rule
+    public ExpectedException exception = ExpectedException.none();
 
     /**
      * Delete is supported
@@ -44,12 +48,11 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
     @Test
     public void testTableWrongType() {
         final MutableTable mutableTable = new MutableTable();
-        try {
-            getUpdateCallback().deleteFrom(mutableTable);
-            fail("Should get an exception that the type of the table is wrong.");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Not an HBase table: " + mutableTable, e.getMessage());
-        }
+
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Not an HBase table: " + mutableTable);
+
+        getUpdateCallback().deleteFrom(mutableTable);
     }
 
     /**
@@ -59,14 +62,11 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testHBaseClientNullAtBuilder() throws IOException {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            new HBaseRowDeletionBuilder(null, existingTable);
-            fail("Should get an exception that hBaseClient can't be null.");
-        } catch (IllegalArgumentException e) {
-            assertEquals("hBaseClient cannot be null", e.getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("hBaseClient cannot be null");
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        new HBaseRowDeletionBuilder(null, existingTable);
     }
 
     /**
@@ -76,14 +76,12 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testNotSettingRowkey() throws IOException {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            getUpdateCallback().deleteFrom(existingTable).execute();
-            fail("Should get an exception that the columnFamily doesn't exist.");
-        } catch (MetaModelException e) {
-            assertEquals("Key cannot be null", e.getMessage());
-        }
+        exception.expect(MetaModelException.class);
+        exception.expectMessage("Key cannot be null");
+
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        getUpdateCallback().deleteFrom(existingTable).execute();
     }
 
     /**
@@ -91,12 +89,10 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreatingTheHBaseClientWithTableNameNull() {
-        try {
-            new HBaseClient(getDataContext().getConnection()).deleteRow(null, new String("1"));
-            fail("Should get an exception that tableName is null");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't delete a row without having tableName or rowKey", e.getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Can't delete a row without having tableName or rowKey");
+
+        new HBaseClient(getDataContext().getConnection()).deleteRow(null, new String("1"));
     }
 
     /**
@@ -104,78 +100,69 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreatingTheHBaseClientWithRowKeyNull() {
-        try {
-            new HBaseClient(getDataContext().getConnection()).deleteRow("tableName", null);
-            fail("Should get an exception that rowKey is null");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't delete a row without having tableName or rowKey", e.getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Can't delete a row without having tableName or rowKey");
+
+        new HBaseClient(getDataContext().getConnection()).deleteRow("tableName", null);
     }
 
     /**
      * Goodflow. Deleting a row, that doesn't exist, should not throw an exception
+     *
+     * @throws IOException
      */
     @Test
-    public void testDeletingNotExistingRow() {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-
-            checkRows(false, false);
-            final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
-                    existingTable);
-            rowDeletionBuilder.setKey(RK_1);
-            rowDeletionBuilder.execute();
-            checkRows(false, false);
-        } catch (Exception e) {
-            fail("Should not get an exception that the row doesn't exist.");
-        }
+    public void testDeletingNotExistingRow() throws IOException {
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+
+        checkRows(false, false);
+        final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
+                existingTable);
+        rowDeletionBuilder.setKey(RK_1);
+        rowDeletionBuilder.execute();
+        checkRows(false, false);
     }
 
     /**
      * Goodflow. Deleting a row, which has an empty rowKey value, should not throw an exception
+     *
+     * @throws IOException
      */
     @Test
-    public void testUsingAnEmptyRowKeyValue() {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-
-            checkRows(false, false);
-            final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
-                    existingTable);
-            rowDeletionBuilder.setKey("");
-            rowDeletionBuilder.execute();
-            checkRows(false, false);
-        } catch (Exception e) {
-            fail("Should not get an exception that the rowkey is empty.");
-        }
+    public void testUsingAnEmptyRowKeyValue() throws IOException {
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+
+        checkRows(false, false);
+        final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
+                existingTable);
+        rowDeletionBuilder.setKey("");
+        rowDeletionBuilder.execute();
+        checkRows(false, false);
     }
 
     /**
      * Goodflow. Deleting a row succesfully.
+     *
+     * @throws IOException
      */
     @Test
-    public void testDeleteRowSuccesfully() {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR, false);
-            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-
-            checkRows(false, false);
-            final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
-            setValuesInInsertionBuilder(row, rowInsertionBuilder);
-            rowInsertionBuilder.execute();
-            checkRows(true, false);
-            final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
-                    existingTable);
-            rowDeletionBuilder.setKey(RK_1);
-            rowDeletionBuilder.execute();
-            checkRows(false, false);
-        } catch (Exception e) {
-            fail("Should not get an exception on deleting a row.");
-        }
+    public void testDeleteRowSuccesfully() throws IOException {
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+
+        checkRows(false, false);
+        final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
+        setValuesInInsertionBuilder(row, rowInsertionBuilder);
+        rowInsertionBuilder.execute();
+        checkRows(true, false);
+        final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback().deleteFrom(
+                existingTable);
+        rowDeletionBuilder.setKey(RK_1);
+        rowDeletionBuilder.execute();
+        checkRows(false, false);
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5313a492/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
index 70ad3f8..e3493e9 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
@@ -25,16 +25,20 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.metamodel.MetaModelException;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 
 public class DropTableTest extends HBaseUpdateCallbackTest {
+    @Rule
+    public ExpectedException exception = ExpectedException.none();
 
     /**
      * Check if drop table is supported
      */
     @Test
     public void testDropTableSupported() {
-            assertTrue(getUpdateCallback().isDropTableSupported());
+        assertTrue(getUpdateCallback().isDropTableSupported());
     }
 
     /**
@@ -42,13 +46,11 @@ public class DropTableTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testDropTableThatDoesntExist() {
-            try {
-                final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-                getUpdateCallback().dropTable(table).execute();
-                fail("Should get an exception that the table doesn't exist in the datastore");
-            } catch (MetaModelException e) {
-                assertEquals("Trying to delete a table that doesn't exist in the datastore.", e.getMessage());
-            }
+        exception.expect(MetaModelException.class);
+        exception.expectMessage("Trying to delete a table that doesn't exist in the datastore.");
+
+        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+        getUpdateCallback().dropTable(table).execute();
     }
 
     /**
@@ -56,29 +58,24 @@ public class DropTableTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreatingTheHBaseClientWithTableNameNull() {
-            try {
-                new HBaseClient(getDataContext().getConnection()).dropTable(null);
-                fail("Should get an exception that tableName is null");
-            } catch (IllegalArgumentException e) {
-                assertEquals("Can't drop a table without having the tableName", e.getMessage());
-            }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Can't drop a table without having the tableName");
+
+        new HBaseClient(getDataContext().getConnection()).dropTable(null);
     }
 
     /**
-     * Goodflow. Droping a table succesfully.
+     * Goodflow. Dropping a table successfully.
+     *
      * @throws IOException
      */
     @Test
     public void testDropTableSuccesfully() throws IOException {
-            try {
-                final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                        CF_BAR);
-                getUpdateCallback().dropTable(existingTable).execute();
-                try (final Admin admin = getDataContext().getAdmin()) {
-                    assertFalse(admin.tableExists(TableName.valueOf(TABLE_NAME)));
-                }
-            } catch (Exception e) {
-                fail("Should not get an exception that the table doesn't exist in the datastore");
-            }
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        getUpdateCallback().dropTable(existingTable).execute();
+        try (final Admin admin = getDataContext().getAdmin()) {
+            assertFalse(admin.tableExists(TableName.valueOf(TABLE_NAME)));
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5313a492/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
index 785fdab..58e60cc 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
@@ -96,7 +96,8 @@ public abstract class HBaseTestCase {
     }
 
     /**
-     * Get's the test configuration file. An example file can be found at the root folder of this project.
+     * Gets the test configuration file. An example file can be found at the root folder of this project.
+     * 
      * @return Location of the configuration file.
      */
     protected String getPropertyFilePath() {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5313a492/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
index 425100f..0d279b8 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
@@ -22,10 +22,13 @@ import static org.junit.Assert.*;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collection;
+import java.util.Arrays;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
@@ -38,13 +41,9 @@ import org.apache.metamodel.schema.Table;
 import org.apache.metamodel.util.SimpleTableDef;
 import org.junit.After;
 import org.junit.Before;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
 
-    private static final Logger logger = LoggerFactory.getLogger(HBaseClient.class);
-
     private HBaseUpdateCallback updateCallback;
     private MutableSchema schema;
 
@@ -63,10 +62,12 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
     }
 
     /**
-     * Drop the table if it exists.
-     * After that check in the schema and the datastore if the actions have been executed succesfully.
+     * Drop the table if it exists. After that check in the schema and the datastore if the actions have been executed
+     * successfully.
+     *
+     * @throws IOException
      */
-    protected void dropTableIfItExists() {
+    protected void dropTableIfItExists() throws IOException {
         if (schema != null) {
             final Table table = schema.getTableByName(TABLE_NAME);
             if (table != null) {
@@ -76,16 +77,14 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
                 // Check in the datastore
                 try (final Admin admin = getDataContext().getAdmin()) {
                     assertFalse(admin.tableExists(TableName.valueOf(TABLE_NAME)));
-                } catch (IOException e) {
-                    fail("Should not an exception checking if the table exists");
                 }
             }
         }
     }
 
     /**
-     * Check if the table has been inserted succesfully.
-     * Checks are performed in the schema and the datastore.
+     * Check if the table has been inserted successfully. Checks are performed in the schema and the datastore.
+     * 
      * @throws IOException because the admin object needs to be created
      */
     protected void checkSuccesfullyInsertedTable() throws IOException {
@@ -94,8 +93,6 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         // Check in the datastore
         try (final Admin admin = getDataContext().getAdmin()) {
             assertTrue(admin.tableExists(TableName.valueOf(TABLE_NAME)));
-        } catch (IOException e) {
-            fail("Should not an exception checking if the table exists");
         }
     }
 
@@ -110,7 +107,7 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
      */
     protected HBaseTable createAndAddTableToDatastore(final String tableName, final String idColumn,
             final String columnFamily1, final String columnFamily2) throws IOException {
-        final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
+        final Set<String> columnFamilies = new LinkedHashSet<>();
         columnFamilies.add(idColumn);
         columnFamilies.add(columnFamily1);
         columnFamilies.add(columnFamily2);
@@ -128,25 +125,25 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
      * @param columnFamily3 columnFamily 3 is not required and can be used to test errors
      * @return created {@link HBaseTable}
      */
-    protected HBaseTable createHBaseTable(final String tableName, final String idColumn, final String columnFamily1,
-            final String columnFamily2, final String columnFamily3) {
+    protected HBaseTable createHBaseTable(final String tableName, final String idColumn,
+            final String... columnFamilies) {
         String[] columnNames;
-        ColumnType[] columnTypes;
 
-        if (idColumn == null && columnFamily3 == null) {
-            columnNames = new String[] { columnFamily1, columnFamily2 };
-            columnTypes = new ColumnType[] { ColumnType.STRING, ColumnType.STRING };
-        } else if (idColumn != null && columnFamily3 == null) {
-            columnNames = new String[] { idColumn, columnFamily1, columnFamily2 };
-            columnTypes = new ColumnType[] { ColumnType.STRING, ColumnType.STRING, ColumnType.STRING };
-        } else if (idColumn == null && columnFamily3 != null) {
-            columnNames = new String[] { columnFamily1, columnFamily2, columnFamily3 };
-            columnTypes = new ColumnType[] { ColumnType.STRING, ColumnType.STRING, ColumnType.STRING };
+        if (idColumn == null) {
+            columnNames = columnFamilies;
         } else {
-            columnNames = new String[] { idColumn, columnFamily1, columnFamily2, columnFamily3 };
-            columnTypes = new ColumnType[] { ColumnType.STRING, ColumnType.STRING, ColumnType.STRING,
-                    ColumnType.STRING };
+            columnNames = new String[columnFamilies.length + 1];
+
+            columnNames[0] = idColumn;
+
+            for (int i = 0; i < columnFamilies.length; i++) {
+                columnNames[i + 1] = columnFamilies[i];
+            }
         }
+
+        ColumnType[] columnTypes = new ColumnType[columnNames.length];
+        Arrays.fill(columnTypes, ColumnType.STRING);
+
         final SimpleTableDef tableDef = new SimpleTableDef(tableName, columnNames, columnTypes);
         return new HBaseTable(getDataContext(), tableDef, schema, ColumnType.STRING);
     }
@@ -160,9 +157,9 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
      * @param qualifiersNull true will create all {@link HBaseColumn}'s with qualifier null
      * @return {@link LinkedHashMap}<{@link HBaseColumn}, {@link Object}>
      */
-    protected static LinkedHashMap<HBaseColumn, Object> createRow(final HBaseTable table, final String idColumn,
+    protected static Map<HBaseColumn, Object> createRow(final HBaseTable table, final String idColumn,
             final String columnFamily1, final String columnFamily2, final boolean qualifiersNull) {
-        final LinkedHashMap<HBaseColumn, Object> map = new LinkedHashMap<>();
+        final Map<HBaseColumn, Object> map = new LinkedHashMap<>();
 
         // Columns
         final ArrayList<HBaseColumn> columns = new ArrayList<>();
@@ -204,10 +201,8 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
      * @param row {@link LinkedHashMap}<{@link HBaseColumn}, {@link Object}>
      * @return {@link List}<{@link HBaseColumn}>
      */
-    protected static List<HBaseColumn> getHBaseColumnsFromRow(final LinkedHashMap<HBaseColumn, Object> row) {
-        final List<HBaseColumn> columns = new ArrayList<>();
-        columns.addAll(row.keySet());
-        return columns;
+    protected static List<HBaseColumn> getHBaseColumnsFromRow(final Map<HBaseColumn, Object> row) {
+        return row.keySet().stream().collect(Collectors.toList());
     }
 
     /**
@@ -216,7 +211,7 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
      * @param rowInsertionBuilder insertionBuilder to be set
      * @param enoughMatchingValues if true, the amount of columns match the amount of values
      */
-    protected void setValuesInInsertionBuilder(final LinkedHashMap<HBaseColumn, Object> row,
+    protected void setValuesInInsertionBuilder(final Map<HBaseColumn, Object> row,
             final HBaseRowInsertionBuilder rowInsertionBuilder) {
         int i = 0;
         for (Object value : row.values()) {
@@ -225,12 +220,6 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         }
     }
 
-    protected Collection<Object> getToLittleValues(final LinkedHashMap<HBaseColumn, Object> row) {
-        Collection<Object> values = row.values();
-        values.remove(V_123_BYTE_ARRAY);
-        return values;
-    }
-
     /**
      * Checks that the table does or doesn't have rows, depending on the rowsExists parameter
      * @param rowsExist true, check that the rows exists. false, check that the result is empty.
@@ -260,19 +249,6 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         }
     }
 
-    /**
-     * Warn that the test(method) of a class is not executed, because the test-file hasn't been set.
-     * See {@link HBaseTestCase#getPropertyFilePath}
-     * @param className
-     * @param methodName
-     */
-    protected void warnAboutANotExecutedTest(String className, String methodName) {
-        String logWarning = "Test \"" + className + "#" + methodName
-                + "()\" is not executed, because the HBasetest is not configured.";
-        System.err.println(logWarning);
-        logger.warn(logWarning);
-    }
-
     protected HBaseUpdateCallback getUpdateCallback() {
         return updateCallback;
     }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/5313a492/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
index 82ac9d8..61e8f9c 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
@@ -23,14 +23,18 @@ import static org.junit.Assert.*;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.schema.MutableTable;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 
 public class InsertRowTest extends HBaseUpdateCallbackTest {
+    @Rule
+    public ExpectedException exception = ExpectedException.none();
 
     /**
      * Check if inserting into a table is supported
@@ -49,14 +53,12 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testOnlyUsingTableParameter() throws IOException {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+        exception.expect(UnsupportedOperationException.class);
+        exception.expectMessage("We need an explicit list of columns when inserting into an HBase table.");
+
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                     CF_BAR);
-            getUpdateCallback().insertInto(existingTable);
-            fail("Should get an exception that this method is not supported");
-        } catch (UnsupportedOperationException e) {
-            assertEquals("We need an explicit list of columns when inserting into an HBase table.", e.getMessage());
-        }
+        getUpdateCallback().insertInto(existingTable);
     }
 
     /**
@@ -67,17 +69,14 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     @Test
     public void testWrongTableType() throws IOException {
         final MutableTable mutableTable = new MutableTable();
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR, false);
-            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-            getUpdateCallback().insertInto(mutableTable, columns);
-            fail("Should get an exception that the type of the table is wrong.");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Not an HBase table: " + mutableTable, e.getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Not an HBase table: " + mutableTable);
+
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+        getUpdateCallback().insertInto(mutableTable, columns);
     }
 
     /**
@@ -87,14 +86,12 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testColumnsNullAtUpdateCallBack() throws IOException {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            getUpdateCallback().insertInto(existingTable, null);
-            fail("Should get an exception that the columns list is null.");
-        } catch (IllegalArgumentException e) {
-            assertEquals("The hbaseColumns list is null or empty", e.getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("The hbaseColumns list is null or empty");
+
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        getUpdateCallback().insertInto(existingTable, null);
     }
 
     /**
@@ -104,14 +101,11 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testColumnsEmptyAtUpdateCallBack() throws IOException {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            getUpdateCallback().insertInto(existingTable, new ArrayList<HBaseColumn>());
-            fail("Should get an exception that the columns list is empty.");
-        } catch (IllegalArgumentException e) {
-            assertEquals("The hbaseColumns list is null or empty", e.getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("The hbaseColumns list is null or empty");
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        getUpdateCallback().insertInto(existingTable, new ArrayList<HBaseColumn>());
     }
 
     /**
@@ -121,20 +115,17 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testTableThatDoesntExist() throws IOException {
-        final HBaseTable wrongTable = createHBaseTable("NewTableNotInSchema", HBaseDataContext.FIELD_ID, "cf1", "cf2",
-                null);
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR, false);
-            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-            getUpdateCallback().insertInto(wrongTable, columns);
-            fail("Should get an exception that the table isn't in the schema.");
-        } catch (MetaModelException e) {
-            assertEquals("Trying to insert data into table: " + wrongTable.getName() + ", which doesn't exist yet", e
-                    .getMessage());
-        }
+        final HBaseTable wrongTable = createHBaseTable("NewTableNotInSchema", HBaseDataContext.FIELD_ID, "cf1", "cf2");
+
+        exception.expect(MetaModelException.class);
+        exception.expectMessage("Trying to insert data into table: " + wrongTable.getName()
+                + ", which doesn't exist yet");
+
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+        getUpdateCallback().insertInto(wrongTable, columns);
     }
 
     /**
@@ -144,16 +135,14 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testIDColumnDoesntExistInColumnsArray() throws IOException {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+        exception.expect(MetaModelException.class);
+        exception.expectMessage("The ID-Column was not found");
+
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                     CF_BAR);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, null, CF_FOO, CF_BAR, false);
-            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-            getUpdateCallback().insertInto(existingTable, columns);
-            fail("Should get an exception that ID-column doesn't exist.");
-        } catch (MetaModelException e) {
-            assertEquals("The ID-Column was not found", e.getMessage());
-        }
+        final Map<HBaseColumn, Object> row = createRow(existingTable, null, CF_FOO, CF_BAR, false);
+        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+        getUpdateCallback().insertInto(existingTable, columns);
     }
 
     /**
@@ -164,20 +153,18 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     @Test
     public void testColumnFamilyDoesntExistsBecauseItsNull() throws IOException {
         final String wrongColumnFamily = "wrongColumnFamily";
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR, false);
-            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-            final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    wrongColumnFamily, null);
-            getUpdateCallback().insertInto(wrongTable, columns);
-            fail("Should get an exception that the columnFamily doesn't exist.");
-        } catch (MetaModelException e) {
-            assertEquals(String.format("ColumnFamily: %s doesn't exist in the schema of the table", wrongColumnFamily),
-                    e.getMessage());
-        }
+
+        exception.expect(MetaModelException.class);
+        exception.expectMessage(String.format("ColumnFamily: %s doesn't exist in the schema of the table",
+                wrongColumnFamily));
+
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+        final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                wrongColumnFamily);
+        getUpdateCallback().insertInto(wrongTable, columns);
     }
 
     /**
@@ -188,20 +175,18 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     @Test
     public void testColumnFamilyDoesntExistsBecauseItsNew() throws IOException {
         final String wrongColumnFamily = "newColumnFamily";
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR, false);
-            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-            final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
-                    wrongColumnFamily);
-            getUpdateCallback().insertInto(wrongTable, columns);
-            fail("Should get an exception that the columnFamily doesn't exist.");
-        } catch (MetaModelException e) {
-            assertEquals(String.format("ColumnFamily: %s doesn't exist in the schema of the table", wrongColumnFamily),
-                    e.getMessage());
-        }
+
+        exception.expect(MetaModelException.class);
+        exception.expectMessage(String.format("ColumnFamily: %s doesn't exist in the schema of the table",
+                wrongColumnFamily));
+
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+        final HBaseTable wrongTable = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
+                wrongColumnFamily);
+        getUpdateCallback().insertInto(wrongTable, columns);
     }
 
     /**
@@ -209,18 +194,15 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreatingTheHBaseClientWithTableNameNull() {
-        try {
-            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
-                    false);
-            final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
-            final Object[] values = new String[] { "Values" };
-            new HBaseClient(getDataContext().getConnection()).insertRow(null, columns, values, 0);
-            fail("Should get an exception that tableName is null");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
-                    .getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage(
+                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
+
+        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
+        final Object[] values = new String[] { "Values" };
+        new HBaseClient(getDataContext().getConnection()).insertRow(null, columns, values, 0);
     }
 
     /**
@@ -228,14 +210,12 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreatingTheHBaseClientWithColumnsNull() {
-        try {
-            final Object[] values = new String[] { "Values" };
-            new HBaseClient(getDataContext().getConnection()).insertRow("tableName", null, values, 0);
-            fail("Should get an exception that columns is null");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
-                    .getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage(
+                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
+
+        final Object[] values = new String[] { "Values" };
+        new HBaseClient(getDataContext().getConnection()).insertRow("tableName", null, values, 0);
     }
 
     /**
@@ -243,17 +223,14 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreatingTheHBaseClientWithValuesNull() {
-        try {
-            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
-                    false);
-            final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
-            new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, null, 0);
-            fail("Should get an exception that values is null");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
-                    .getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage(
+                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
+
+        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
+        new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, null, 0);
     }
 
     /**
@@ -261,18 +238,15 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreatingTheHBaseClientWithIndexOfIdColumnOutOfBounce() {
-        try {
-            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
-                    false);
-            final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
-            final Object[] values = new String[] { "Values" };
-            new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 10);
-            fail("Should get an exception that the indexOfIdColumn is incorrect");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
-                    .getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage(
+                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
+
+        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
+        final Object[] values = new String[] { "Values" };
+        new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 10);
     }
 
     /**
@@ -280,102 +254,93 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
      */
     @Test
     public void testCreatingTheHBaseClientWithRowKeyNull() {
-        try {
-            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR,
-                    false);
-            final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
-            final Object[] values = new String[] { null };
-            new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 0);
-            fail("Should get an exception that the indexOfIdColumn is incorrect");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn", e
-                    .getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage(
+                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
+
+        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
+        final Object[] values = new String[] { null };
+        new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 0);
     }
 
     /**
      * Inserting a row without setting enough values directly on the HBaseClient, should throw exception.
      * NOTE: This exception is already prevented when using the {@link HBaseRowInsertionBuilder}
-     * @throws IOException 
+     * @throws IOException
      */
     @Test
     public void testNotSettingEnoughValues() throws IOException {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR, false);
-            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-            final Collection<Object> values = getToLittleValues(row);
-            final HBaseClient hBaseClient = ((HBaseDataContext) getUpdateCallback().getDataContext()).getHBaseClient();
-            hBaseClient.insertRow(TABLE_NAME, columns.toArray(new HBaseColumn[columns.size()]), values.toArray(
-                    new Object[values.size()]), 0); // TODO: find the ID-column
-            fail("Should get an exception when insering directly into the HBaseClient without having enough values.");
-        } catch (IllegalArgumentException e) {
-            assertEquals("The amount of columns don't match the amount of values", e.getMessage());
-        }
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("The amount of columns don't match the amount of values");
+
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+        final Collection<Object> values = getTooLittleValues(row);
+        final HBaseClient hBaseClient = ((HBaseDataContext) getUpdateCallback().getDataContext()).getHBaseClient();
+        hBaseClient.insertRow(TABLE_NAME, columns.toArray(new HBaseColumn[columns.size()]), values.toArray(
+                new Object[values.size()]), 0); // TODO: find the ID-column
+    }
+
+    private Collection<Object> getTooLittleValues(final Map<HBaseColumn, Object> row) {
+        Collection<Object> values = row.values();
+        values.remove(V_123_BYTE_ARRAY);
+        return values;
     }
 
     /**
      * Goodflow. Using an existing table and columns, should work
+     *
+     * @throws IOException
      */
     @Test
-    public void testInsertIntoWithoutExecute() {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR, false);
-            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-            getUpdateCallback().insertInto(existingTable, columns);
-        } catch (Exception e) {
-            fail("No exception should be thrown, when inserting into an existing table.");
-        }
+    public void testInsertIntoWithoutExecute() throws IOException {
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+        getUpdateCallback().insertInto(existingTable, columns);
     }
 
     /**
      * Goodflow, creating a row with qualifiers null should work.
+     *
+     * @throws IOException
      */
     @Test
-    public void testQaulifierNull() {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR, true);
-            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-
-            checkRows(false, true);
-            final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
-            setValuesInInsertionBuilder(row, rowInsertionBuilder);
-            rowInsertionBuilder.execute();
-            checkRows(true, true);
-        } catch (Exception e) {
-            fail("Inserting a row without qualifiers should work.");
-        }
+    public void testQaulifierNull() throws IOException {
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, true);
+        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+
+        checkRows(false, true);
+        final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
+        setValuesInInsertionBuilder(row, rowInsertionBuilder);
+        rowInsertionBuilder.execute();
+        checkRows(true, true);
     }
 
     /**
      * Goodflow. Inserting a row succesfully (with values set)
+     *
+     * @throws IOException
      */
     @Test
-    public void testInsertingSuccesfully() {
-        try {
-            final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR);
-            final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO,
-                    CF_BAR, false);
-            final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
-
-            checkRows(false, false);
-            final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
-            setValuesInInsertionBuilder(row, rowInsertionBuilder);
-            rowInsertionBuilder.execute();
-            checkRows(true, false);
-        } catch (Exception e) {
-            fail("No exception should be thrown, when inserting with values.");
-        }
+    public void testInsertingSuccesfully() throws IOException {
+        final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final List<HBaseColumn> columns = getHBaseColumnsFromRow(row);
+
+        checkRows(false, false);
+        final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable, columns);
+        setValuesInInsertionBuilder(row, rowInsertionBuilder);
+        rowInsertionBuilder.execute();
+        checkRows(true, false);
     }
 
     /**


[09/31] metamodel git commit: Removed table exists check. Let's assume the createTable message throws an exception when it fails to create the table.

Posted by ka...@apache.org.
Removed table exists check. Let's assume the createTable message throws an exception when it fails to create the table.


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/86084869
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/86084869
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/86084869

Branch: refs/heads/master
Commit: 86084869ab7eefb92d983b472fb4664f3a0f742d
Parents: ea7356a
Author: arjansh <Ar...@humaninference.com>
Authored: Fri May 25 13:12:36 2018 +0200
Committer: arjansh <Ar...@humaninference.com>
Committed: Fri May 25 13:12:36 2018 +0200

----------------------------------------------------------------------
 hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/86084869/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
index 1e957fc..ed7b17c 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
@@ -142,10 +142,6 @@ public final class HBaseClient {
                 }
             }
             admin.createTable(tableDescriptor);
-            final HTableDescriptor[] tables = admin.listTables();
-            if (tables.length != 1 && Bytes.equals(hBasetableName.getName(), tables[0].getTableName().getName())) {
-                throw new IOException("Failed create of table");
-            }
         }
     }
 


[26/31] metamodel git commit: - Refactored contructing String representation of byte arrays. - Have the HBaseRowInsertionBuilder no longer extend the AbstractRowInsertionBuilder, because it does things structurally different in some parts.

Posted by ka...@apache.org.
- Refactored contructing String representation of byte arrays.
- Have the HBaseRowInsertionBuilder no longer extend the AbstractRowInsertionBuilder, because it does things structurally different in some parts.


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/1dc52f64
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/1dc52f64
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/1dc52f64

Branch: refs/heads/master
Commit: 1dc52f64311433d597ba87220eca54da0e85d942
Parents: 0ca1fd2
Author: Arjan Seijkens <a....@quadient.com>
Authored: Fri Jun 15 14:33:19 2018 +0200
Committer: Arjan Seijkens <a....@quadient.com>
Committed: Fri Jun 15 14:33:19 2018 +0200

----------------------------------------------------------------------
 .../apache/metamodel/hbase/HBaseFamilyMap.java  |   6 +-
 .../hbase/HBaseRowInsertionBuilder.java         | 139 ++++++++++++++-----
 .../apache/metamodel/hbase/InsertRowTest.java   |  14 ++
 3 files changed, 126 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/1dc52f64/hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java
index 7483c63..32656c2 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java
@@ -23,6 +23,8 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.Set;
 
+import org.apache.hadoop.hbase.util.Bytes;
+
 public class HBaseFamilyMap implements Map<Object, Object> {
 
     private final NavigableMap<byte[], byte[]> _map;
@@ -105,9 +107,9 @@ public class HBaseFamilyMap implements Map<Object, Object> {
             if (sb.length() > 1) {
                 sb.append(',');
             }
-            sb.append(new String(entry.getKey()));
+            sb.append(Bytes.toString(entry.getKey()));
             sb.append('=');
-            sb.append(new String(entry.getValue()));
+            sb.append(Bytes.toString(entry.getValue()));
         }
         sb.append('}');
         return sb.toString();

http://git-wip-us.apache.org/repos/asf/metamodel/blob/1dc52f64/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
index abbbfee..fe6e07a 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
@@ -27,32 +27,39 @@ import java.util.Set;
 import java.util.stream.Collectors;
 
 import org.apache.metamodel.MetaModelException;
+import org.apache.metamodel.data.DefaultRow;
+import org.apache.metamodel.data.Row;
+import org.apache.metamodel.data.SimpleDataSetHeader;
 import org.apache.metamodel.data.Style;
-import org.apache.metamodel.insert.AbstractRowInsertionBuilder;
 import org.apache.metamodel.insert.RowInsertionBuilder;
+import org.apache.metamodel.query.SelectItem;
 import org.apache.metamodel.schema.Column;
 
 /**
  * A builder-class to insert rows in a HBase datastore.
  */
-public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
-    private List<HBaseColumn> columns = new ArrayList<>();
-    private List<Object> values = new ArrayList<>();
+public class HBaseRowInsertionBuilder implements RowInsertionBuilder {
+    private List<HBaseColumn> _columns = new ArrayList<>();
+    private List<Object> _values = new ArrayList<>();
 
     private int _indexOfIdColumn = -1;
 
+    private final HBaseUpdateCallback _updateCallback;
+    private final HBaseTable _table;
+
     /**
      * Creates a {@link HBaseRowInsertionBuilder}. The table and the column's columnFamilies are checked to exist in the schema.
      * @param updateCallback
      * @param table
-     * @param columns
+     * @param _columns
      * @throws IllegalArgumentException the columns list can't be null or empty
      * @throws MetaModelException when no ID-column is found.
      */
     public HBaseRowInsertionBuilder(final HBaseUpdateCallback updateCallback, final HBaseTable table) {
-        super(updateCallback, table);
+        _updateCallback = updateCallback;
+        _table = table;
 
-        checkTable(updateCallback, table);
+        checkTable(table);
     }
 
     /**
@@ -62,8 +69,11 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
      * @param tableGettingInserts
      * @throws MetaModelException If the table or the columnFamilies don't exist
      */
-    private void checkTable(final HBaseUpdateCallback updateCallback, final HBaseTable tableGettingInserts) {
-        final HBaseTable tableInSchema = (HBaseTable) updateCallback.getDataContext().getDefaultSchema().getTableByName(
+    private void checkTable(final HBaseTable tableGettingInserts) {
+        final HBaseTable tableInSchema = (HBaseTable) _updateCallback
+                .getDataContext()
+                .getDefaultSchema()
+                .getTableByName(
                 tableGettingInserts.getName());
         if (tableInSchema == null) {
             throw new MetaModelException("Trying to insert data into table: " + tableGettingInserts.getName()
@@ -113,20 +123,18 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
         }
 
         // The columns parameter should match the table's columns, just to be sure, this is checked again
-        checkColumnFamilies((HBaseTable) getTable(), getColumnFamilies(getColumns()));
+        checkColumnFamilies(getTable(), getColumnFamilies(getColumns()));
 
-        ((HBaseDataContext) getUpdateCallback().getDataContext()).getHBaseClient().insertRow(getTable().getName(),
+        ((HBaseDataContext) _updateCallback.getDataContext()).getHBaseClient().insertRow(getTable().getName(),
                 getColumns(), getValues(), _indexOfIdColumn);
     }
 
-    @Override
-    protected HBaseColumn[] getColumns() {
-        return columns.toArray(new HBaseColumn[columns.size()]);
+    private HBaseColumn[] getColumns() {
+        return _columns.toArray(new HBaseColumn[_columns.size()]);
     }
 
-    @Override
-    protected Object[] getValues() {
-        return values.toArray(new Object[values.size()]);
+    private Object[] getValues() {
+        return _values.toArray(new Object[_values.size()]);
     }
 
     @Override
@@ -137,19 +145,19 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
 
         final HBaseColumn hbaseColumn = getHbaseColumn(column);
 
-        for (int i = 0; i < columns.size(); i++) {
-            if (columns.get(i).equals(hbaseColumn)) {
-                values.set(i, value);
+        for (int i = 0; i < _columns.size(); i++) {
+            if (_columns.get(i).equals(hbaseColumn)) {
+                _values.set(i, value);
                 return this;
             }
         }
 
         if (hbaseColumn.isPrimaryKey()) {
-            _indexOfIdColumn = columns.size();
+            _indexOfIdColumn = _columns.size();
         }
 
-        columns.add((HBaseColumn) hbaseColumn);
-        values.add(value);
+        _columns.add((HBaseColumn) hbaseColumn);
+        _values.add(value);
 
         return this;
     }
@@ -171,30 +179,99 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
     }
 
     @Override
+    public boolean isSet(final Column column) {
+        for (int i = 0; i < _columns.size(); i++) {
+            if (_columns.get(i).equals(column)) {
+                return _values.get(i) != null;
+            }
+        }
+        return false;
+    }
+
+    @Override
     public RowInsertionBuilder value(final int columnIndex, final Object value) {
-        values.set(columnIndex, value);
+        return value(columnIndex, value, null);
+    }
+
+    @Override
+    public RowInsertionBuilder value(int columnIndex, Object value, Style style) {
+        _values.set(columnIndex, value);
         return this;
     }
 
     @Override
     public RowInsertionBuilder value(final String columnName, final Object value) {
-        for (Column column : columns) {
+        return value(columnName, value, null);
+    }
+
+    @Override
+    public RowInsertionBuilder value(Column column, Object value) {
+        return value(column, value, null);
+    }
+
+    @Override
+    public RowInsertionBuilder value(String columnName, Object value, Style style) {
+        for (Column column : _columns) {
             if (column.getName().equals(columnName)) {
                 return value(column, value, null);
             }
         }
 
         throw new IllegalArgumentException("No such column in table: " + columnName + ", available columns are: "
-                + columns);
+                + _columns);
     }
 
     @Override
-    public boolean isSet(final Column column) {
-        for (int i = 0; i < columns.size(); i++) {
-            if (columns.get(i).equals(column)) {
-                return values.get(i) != null;
+    public Row toRow() {
+        return new DefaultRow(new SimpleDataSetHeader(_columns.stream().map(SelectItem::new).collect(Collectors
+                .toList())), getValues());
+    }
+
+    @Override
+    public HBaseTable getTable() {
+        return _table;
+    }
+
+    @Override
+    public RowInsertionBuilder like(Row row) {
+        List<SelectItem> selectItems = row.getSelectItems();
+        for (int i = 0; i < selectItems.size(); i++) {
+            SelectItem selectItem = selectItems.get(i);
+            Column column = selectItem.getColumn();
+            if (column != null) {
+                if (_table == column.getTable()) {
+                    value(column, row.getValue(i));
+                } else {
+                    value(column.getName(), row.getValue(i));
+                }
             }
         }
-        return false;
+        return this;
+    }
+
+    @Override
+    public String toSql() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("INSERT INTO ");
+        sb.append(_table.getQualifiedLabel());
+        sb.append("(");
+        sb.append(_columns.stream().map(Column::getName).collect(Collectors.joining(",")));
+        sb.append(") VALUES (");
+        sb.append(_values.stream().map(value -> {
+            if (value == null) {
+                return "NULL";
+            } else if (value instanceof String) {
+                return "\"" + value + "\"";
+            } else {
+                return value.toString();
+            }
+        }).collect(Collectors.joining(",")));
+        sb.append(")");
+        return sb.toString();
+    }
+
+    @Override
+    public String toString() {
+        return toSql();
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/1dc52f64/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
index 31a33d0..1c21cba 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
@@ -245,6 +245,20 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
         checkRows(true, false);
     }
 
+    @Test
+    public void testSqlRepresentation() throws IOException {
+        final HBaseTable table = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+
+        RowInsertionBuilder insertBuilder = getUpdateCallback()
+                .insertInto(table)
+                .value(new HBaseColumn(CF_FOO, Q_BAH, table), V_WORLD)
+                .value(new HBaseColumn(CF_FOO, Q_HELLO, table), V_THERE)
+                .value(new HBaseColumn(CF_BAR, Q_HEY, table), V_YO);
+
+        assertEquals("INSERT INTO HBase.table_for_junit(foo:bah,foo:hello,bar:hey) "
+                + "VALUES (\"world\",\"there\",\"yo\")", insertBuilder.toSql());
+    }
+
     /**
      * Converts a list of {@link HBaseColumn}'s to an array of {@link HBaseColumn}'s
      *


[28/31] metamodel git commit: HBase improvements - Small refactoring on loging

Posted by ka...@apache.org.
HBase improvements - Small refactoring on loging


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/9f402956
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/9f402956
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/9f402956

Branch: refs/heads/master
Commit: 9f402956cf483b6984cfcd02994aabb253c349bd
Parents: 738a97d
Author: Gerard Dellemann <g....@quadient.com>
Authored: Tue Jun 19 13:12:39 2018 +0200
Committer: Gerard Dellemann <g....@quadient.com>
Committed: Tue Jun 19 13:12:39 2018 +0200

----------------------------------------------------------------------
 .../main/java/org/apache/metamodel/hbase/HBaseClient.java    | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/9f402956/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
index e73a643..ecf5e4c 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
@@ -86,11 +86,11 @@ final class HBaseClient {
                     }
                     final byte[] value = getValueAsByteArray(values[i]);
                     // A NULL value, doesn't get inserted in HBase
-                    // TODO: Do we delete the cell (and therefore the qualifier) if the table get's updated?
+                    // TODO: Do we delete the cell (and therefore the qualifier) if the table get's updated by a NULL
+                    // value?
                     if (value == null) {
-                        logger.warn("The value of column '{}:{}' is null. This insertion is skipped", columns[i]
-                                .getColumnFamily()
-                                .toString(), columns[i].getQualifier().toString());
+                        logger.info("The value of column '{}' is null. This insertion is skipped", columns[i]
+                                .getName());
                     } else {
                         put.addColumn(columnFamily, qualifier, value);
                     }


[27/31] metamodel git commit: HBase Improvement - Fixed a bug for inserting NULL values and refactored some tests

Posted by ka...@apache.org.
HBase Improvement - Fixed a bug for inserting NULL values and refactored some tests


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/738a97d0
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/738a97d0
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/738a97d0

Branch: refs/heads/master
Commit: 738a97d042ffffcfb00db71b5199a0bc0663432a
Parents: 1dc52f6
Author: Gerard Dellemann <g....@quadient.com>
Authored: Tue Jun 19 11:27:22 2018 +0200
Committer: Gerard Dellemann <g....@quadient.com>
Committed: Tue Jun 19 11:27:22 2018 +0200

----------------------------------------------------------------------
 .../org/apache/metamodel/hbase/HBaseClient.java |  18 ++-
 .../hbase/HBaseRowDeletionBuilder.java          |   6 +-
 .../apache/metamodel/hbase/DeleteRowTest.java   |   7 +-
 .../apache/metamodel/hbase/DropTableTest.java   |  14 +--
 .../apache/metamodel/hbase/HBaseClientTest.java | 102 +++++++++++++--
 .../hbase/HBaseUpdateCallbackTest.java          |  10 ++
 .../apache/metamodel/hbase/InsertRowTest.java   | 123 ++++++-------------
 7 files changed, 158 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/738a97d0/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
index 0594a5b..e73a643 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
@@ -76,8 +76,8 @@ final class HBaseClient {
                 if (i != indexOfIdColumn) {
                     // NullChecker is already forced within the HBaseColumn class
                     final byte[] columnFamily = Bytes.toBytes(columns[i].getColumnFamily());
-                    // An HBaseColumn doesn't need a qualifier, this only works when the qualifier is empty (not null).
-                    // Otherwise NullPointer exceptions will happen
+                    // An HBaseColumn doesn't need a qualifier, this only works when the qualifier is empty (not
+                    // null). Otherwise NullPointer exceptions will happen
                     byte[] qualifier = null;
                     if (columns[i].getQualifier() != null) {
                         qualifier = Bytes.toBytes(columns[i].getQualifier());
@@ -85,7 +85,15 @@ final class HBaseClient {
                         qualifier = Bytes.toBytes(new String(""));
                     }
                     final byte[] value = getValueAsByteArray(values[i]);
-                    put.addColumn(columnFamily, qualifier, value);
+                    // A NULL value, doesn't get inserted in HBase
+                    // TODO: Do we delete the cell (and therefore the qualifier) if the table get's updated?
+                    if (value == null) {
+                        logger.warn("The value of column '{}:{}' is null. This insertion is skipped", columns[i]
+                                .getColumnFamily()
+                                .toString(), columns[i].getQualifier().toString());
+                    } else {
+                        put.addColumn(columnFamily, qualifier, value);
+                    }
                 }
             }
             // Add the put to the table
@@ -187,7 +195,9 @@ final class HBaseClient {
      */
     private byte[] getValueAsByteArray(final Object value) {
         byte[] valueAsByteArray;
-        if (value instanceof byte[]) {
+        if (value == null) {
+            valueAsByteArray = null;
+        } else if (value instanceof byte[]) {
             valueAsByteArray = (byte[]) value;
         } else {
             valueAsByteArray = Bytes.toBytes(value.toString());

http://git-wip-us.apache.org/repos/asf/metamodel/blob/738a97d0/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
index 078fe5b..e8afda8 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
@@ -40,14 +40,14 @@ public class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
 
     /**
      * Creates a {@link HBaseRowDeletionBuilder}
-     * @param hBaseWriter
+     * @param dataContext
      * @param table
-     * @throws IllegalArgumentException when the hBaseWriter is null
+     * @throws IllegalArgumentException when the dataContext is null
      */
     public HBaseRowDeletionBuilder(final HBaseDataContext dataContext, final Table table) {
         super(table);
         if (dataContext == null) {
-            throw new IllegalArgumentException("hBaseClient cannot be null");
+            throw new IllegalArgumentException("DataContext cannot be null");
         }
         this._dataContext = dataContext;
     }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/738a97d0/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
index 4e588ca..65a8431 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
@@ -55,14 +55,14 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
     }
 
     /**
-     * Creating a HBaseRowDeletionBuilder with the hBaseClient null, should throw an exception
+     * Creating a HBaseRowDeletionBuilder with the DataContext null, should throw an exception
      *
      * @throws IOException
      */
     @Test
-    public void testHBaseClientNullAtBuilder() throws IOException {
+    public void testDataContextNullAtBuilder() throws IOException {
         exception.expect(IllegalArgumentException.class);
-        exception.expectMessage("hBaseClient cannot be null");
+        exception.expectMessage("DataContext cannot be null");
         final HBaseTable existingTable = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
                 CF_BAR);
         new HBaseRowDeletionBuilder(null, existingTable);
@@ -83,7 +83,6 @@ public class DeleteRowTest extends HBaseUpdateCallbackTest {
         getUpdateCallback().deleteFrom(existingTable).execute();
     }
 
-
     /**
      * Goodflow. Deleting a row, that doesn't exist, should not throw an exception
      *

http://git-wip-us.apache.org/repos/asf/metamodel/blob/738a97d0/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
index e3493e9..c94febc 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
@@ -18,7 +18,8 @@
  */
 package org.apache.metamodel.hbase;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 
@@ -54,17 +55,6 @@ public class DropTableTest extends HBaseUpdateCallbackTest {
     }
 
     /**
-     * Creating a HBaseClient with the tableName null, should throw a exception
-     */
-    @Test
-    public void testCreatingTheHBaseClientWithTableNameNull() {
-        exception.expect(IllegalArgumentException.class);
-        exception.expectMessage("Can't drop a table without having the tableName");
-
-        new HBaseClient(getDataContext().getConnection()).dropTable(null);
-    }
-
-    /**
      * Goodflow. Dropping a table successfully.
      *
      * @throws IOException

http://git-wip-us.apache.org/repos/asf/metamodel/blob/738a97d0/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java
index 31d6a0d..f46fd9e 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java
@@ -19,6 +19,7 @@
 package org.apache.metamodel.hbase;
 
 import java.util.LinkedHashSet;
+import java.util.Map;
 import java.util.Set;
 
 import org.junit.Before;
@@ -26,7 +27,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
-public class HBaseClientTest extends HBaseTestCase {
+public class HBaseClientTest extends HBaseUpdateCallbackTest {
     @Rule
     public ExpectedException exception = ExpectedException.none();
 
@@ -36,11 +37,75 @@ public class HBaseClientTest extends HBaseTestCase {
         super.setUp();
     }
 
+    /* Inserting a row */
+
+    /**
+     * Inserting a row with the columns null, should throw a exception
+     */
+    @Test
+    public void testInsertRowWithColumnsNull() {
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage(
+                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
+
+        final Object[] values = new String[] { "Values" };
+        new HBaseClient(getDataContext().getConnection()).insertRow("tableName", null, values, 0);
+    }
+
+    /**
+     * Inserting a row with with the values null, should throw a exception
+     */
+    @Test
+    public void testInsertRowWithValuesNull() {
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage(
+                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
+
+        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
+        new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, null, 0);
+    }
+
     /**
-     * Creating a HBaseClient with the tableName null, should throw a exception
+     * Inserting a row with with the indexOfIdColumn out of bounce, should throw a exception
      */
     @Test
-    public void testCreatingTheHBaseClientWithTableNameNull() {
+    public void testInsertRowWithIndexOfIdColumnOutOfBounce() {
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage(
+                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
+
+        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
+        final Object[] values = new String[] { "Values" };
+        new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 10);
+    }
+
+    /**
+     * Inserting a row with with the rowKey null, should throw a exception
+     */
+    @Test
+    public void testInsertRowWithRowKeyNull() {
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage(
+                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
+
+        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+        final Map<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
+        final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
+        final Object[] values = new String[] { null };
+        new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 0);
+    }
+
+    /* Creating a table */
+
+    /**
+     * Creating a table with the tableName null, should throw a exception
+     */
+    @Test
+    public void testCreateTableWithTableNameNull() {
         exception.expect(IllegalArgumentException.class);
         exception.expectMessage("Can't create a table without having the tableName or columnFamilies");
 
@@ -50,10 +115,10 @@ public class HBaseClientTest extends HBaseTestCase {
     }
 
     /**
-     * Creating a HBaseClient with the tableName null, should throw a exception
+     * Creating a table with the columnFamilies null, should throw a exception
      */
     @Test
-    public void testCreatingTheHBaseClientWithColumnFamiliesNull() {
+    public void testCreateTableWithColumnFamiliesNull() {
         exception.expect(IllegalArgumentException.class);
         exception.expectMessage("Can't create a table without having the tableName or columnFamilies");
 
@@ -61,10 +126,10 @@ public class HBaseClientTest extends HBaseTestCase {
     }
 
     /**
-     * Creating a HBaseClient with the tableName null, should throw a exception
+     * Creating a table with the columnFamilies empty, should throw a exception
      */
     @Test
-    public void testCreatingTheHBaseClientWithColumnFamiliesEmpty() {
+    public void testCreateTableWithColumnFamiliesEmpty() {
         exception.expect(IllegalArgumentException.class);
         exception.expectMessage("Can't create a table without having the tableName or columnFamilies");
 
@@ -72,11 +137,13 @@ public class HBaseClientTest extends HBaseTestCase {
         new HBaseClient(getDataContext().getConnection()).createTable("1", columnFamilies);
     }
 
+    /* Deleting a row */
+
     /**
-     * Creating a HBaseClient with the tableName null, should throw a exception
+     * Deleting a row with the tableName null, should throw a exception
      */
     @Test
-    public void testDeleteRowWithoutTableName() {
+    public void testDeleteRowWithTableNameNull() {
         exception.expect(IllegalArgumentException.class);
         exception.expectMessage("Can't delete a row without having tableName or rowKey");
 
@@ -84,13 +151,26 @@ public class HBaseClientTest extends HBaseTestCase {
     }
 
     /**
-     * Creating a HBaseClient with the rowKey null, should throw a exception
+     * Deleting a row with the rowKey null, should throw a exception
      */
     @Test
-    public void testCreatingTheHBaseClientWithRowKeyNull() {
+    public void testDeleteRowWithRowKeyNull() {
         exception.expect(IllegalArgumentException.class);
         exception.expectMessage("Can't delete a row without having tableName or rowKey");
 
         new HBaseClient(getDataContext().getConnection()).deleteRow("tableName", null);
     }
+
+    /* Dropping/deleting a table */
+
+    /**
+     * Dropping a table with the tableName null, should throw a exception
+     */
+    @Test
+    public void testDropTableWithTableNameNull() {
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Can't drop a table without having the tableName");
+
+        new HBaseClient(getDataContext().getConnection()).dropTable(null);
+    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/738a97d0/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
index 7d03641..403f95d 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
@@ -241,6 +241,16 @@ public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
         }
     }
 
+    /**
+     * Converts a list of {@link HBaseColumn}'s to an array of {@link HBaseColumn}'s
+     *
+     * @param columns
+     * @return Array of {@link HBaseColumn}
+     */
+    protected static HBaseColumn[] convertToHBaseColumnsArray(List<HBaseColumn> columns) {
+        return columns.toArray(new HBaseColumn[columns.size()]);
+    }
+
     protected HBaseUpdateCallback getUpdateCallback() {
         return updateCallback;
     }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/738a97d0/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
index 1c21cba..b380d33 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/InsertRowTest.java
@@ -25,6 +25,10 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.insert.RowInsertionBuilder;
 import org.apache.metamodel.schema.MutableTable;
@@ -96,82 +100,6 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     }
 
     /**
-     * Creating a HBaseClient with the tableName null, should throw a exception
-     */
-    @Test
-    public void testCreatingTheHBaseClientWithTableNameNull() {
-        exception.expect(IllegalArgumentException.class);
-        exception.expectMessage(
-                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
-
-        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
-        final Map<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
-        final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
-        final Object[] values = new String[] { "Values" };
-        new HBaseClient(getDataContext().getConnection()).insertRow(null, columns, values, 0);
-    }
-
-    /**
-     * Creating a HBaseClient with the columns null, should throw a exception
-     */
-    @Test
-    public void testCreatingTheHBaseClientWithColumnsNull() {
-        exception.expect(IllegalArgumentException.class);
-        exception.expectMessage(
-                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
-
-        final Object[] values = new String[] { "Values" };
-        new HBaseClient(getDataContext().getConnection()).insertRow("tableName", null, values, 0);
-    }
-
-    /**
-     * Creating a HBaseClient with the values null, should throw a exception
-     */
-    @Test
-    public void testCreatingTheHBaseClientWithValuesNull() {
-        exception.expect(IllegalArgumentException.class);
-        exception.expectMessage(
-                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
-
-        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
-        final Map<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
-        final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
-        new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, null, 0);
-    }
-
-    /**
-     * Creating a HBaseClient with the indexOfIdColumn out of bounce, should throw a exception
-     */
-    @Test
-    public void testCreatingTheHBaseClientWithIndexOfIdColumnOutOfBounce() {
-        exception.expect(IllegalArgumentException.class);
-        exception.expectMessage(
-                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
-
-        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
-        final Map<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
-        final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
-        final Object[] values = new String[] { "Values" };
-        new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 10);
-    }
-
-    /**
-     * Creating a HBaseClient with the rowKey null, should throw a exception
-     */
-    @Test
-    public void testCreatingTheHBaseClientWithRowKeyNull() {
-        exception.expect(IllegalArgumentException.class);
-        exception.expectMessage(
-                "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
-
-        final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
-        final Map<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, false);
-        final HBaseColumn[] columns = convertToHBaseColumnsArray(getHBaseColumnsFromRow(row));
-        final Object[] values = new String[] { null };
-        new HBaseClient(getDataContext().getConnection()).insertRow(table.getName(), columns, values, 0);
-    }
-
-    /**
      * Inserting a row without setting enough values directly on the HBaseClient, should throw exception.
      * NOTE: This exception is already prevented when using the {@link HBaseRowInsertionBuilder}
      * @throws IOException
@@ -198,6 +126,34 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
     }
 
     /**
+     * Inserting a row with with a value null, should get skipped
+     * @throws IOException 
+     */
+    @Test
+    public void testInsertRowWithValueNull() throws IOException {
+        final HBaseTable table = createAndAddTableToDatastore(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+
+        RowInsertionBuilder insertBuilder = getUpdateCallback()
+                .insertInto(table)
+                .value(new HBaseColumn(HBaseDataContext.FIELD_ID, null, table), RK_1)
+                .value(new HBaseColumn(CF_FOO, Q_BAH, table), V_WORLD)
+                .value(new HBaseColumn(CF_FOO, Q_HELLO, table), null)
+                .value(new HBaseColumn(CF_BAR, Q_HEY, table), V_YO);
+        insertBuilder.execute();
+
+        try (org.apache.hadoop.hbase.client.Table hBaseTable = getDataContext().getConnection().getTable(TableName
+                .valueOf(TABLE_NAME))) {
+            final Get get = new Get(Bytes.toBytes(RK_1));
+            final Result result = hBaseTable.get(get);
+
+            assertFalse(result.isEmpty());
+            assertEquals(V_WORLD, new String(result.getValue(Bytes.toBytes(CF_FOO), Bytes.toBytes(Q_BAH))));
+            assertNull(result.getValue(Bytes.toBytes(CF_FOO), Bytes.toBytes(Q_HELLO)));
+            assertEquals(V_YO, new String(result.getValue(Bytes.toBytes(CF_BAR), Bytes.toBytes(Q_HEY))));
+        }
+    }
+
+    /**
      * Goodflow. Using an existing table and columns, should work
      *
      * @throws IOException
@@ -251,21 +207,12 @@ public class InsertRowTest extends HBaseUpdateCallbackTest {
 
         RowInsertionBuilder insertBuilder = getUpdateCallback()
                 .insertInto(table)
+                .value(new HBaseColumn(HBaseDataContext.FIELD_ID, null, table), RK_1)
                 .value(new HBaseColumn(CF_FOO, Q_BAH, table), V_WORLD)
                 .value(new HBaseColumn(CF_FOO, Q_HELLO, table), V_THERE)
                 .value(new HBaseColumn(CF_BAR, Q_HEY, table), V_YO);
 
-        assertEquals("INSERT INTO HBase.table_for_junit(foo:bah,foo:hello,bar:hey) "
-                + "VALUES (\"world\",\"there\",\"yo\")", insertBuilder.toSql());
-    }
-
-    /**
-     * Converts a list of {@link HBaseColumn}'s to an array of {@link HBaseColumn}'s
-     *
-     * @param columns
-     * @return Array of {@link HBaseColumn}
-     */
-    private static HBaseColumn[] convertToHBaseColumnsArray(List<HBaseColumn> columns) {
-        return columns.toArray(new HBaseColumn[columns.size()]);
+        assertEquals("INSERT INTO HBase.table_for_junit(_id,foo:bah,foo:hello,bar:hey) "
+                + "VALUES (\"junit1\",\"world\",\"there\",\"yo\")", insertBuilder.toSql());
     }
 }


[31/31] metamodel git commit: Empty commit to trigger rebuild

Posted by ka...@apache.org.
Empty commit to trigger rebuild


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/1d67d308
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/1d67d308
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/1d67d308

Branch: refs/heads/master
Commit: 1d67d30867ce78dbabef9e97330b4697a1bee306
Parents: c28a817
Author: Gerard Dellemann <g....@quadient.com>
Authored: Wed Jun 20 10:10:17 2018 +0200
Committer: Gerard Dellemann <g....@quadient.com>
Committed: Wed Jun 20 10:10:17 2018 +0200

----------------------------------------------------------------------

----------------------------------------------------------------------



[11/31] metamodel git commit: HBase Improvements - did some refactoring, added comments, added unittest

Posted by ka...@apache.org.
HBase Improvements - did some refactoring, added comments, added unittest


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/9ab30040
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/9ab30040
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/9ab30040

Branch: refs/heads/master
Commit: 9ab30040e3311f5c9947615470ad1d366de92c0f
Parents: 8608486
Author: Gerard Dellemann <g....@quadient.com>
Authored: Tue May 29 15:28:27 2018 +0200
Committer: Gerard Dellemann <g....@quadient.com>
Committed: Tue May 29 15:28:27 2018 +0200

----------------------------------------------------------------------
 .../org/apache/metamodel/hbase/HBaseClient.java |  92 ++--
 .../org/apache/metamodel/hbase/HBaseColumn.java | 165 +++++--
 .../hbase/HBaseCreateTableBuilder.java          |  10 +-
 .../hbase/HBaseRowDeletionBuilder.java          |  21 +-
 .../hbase/HBaseRowInsertionBuilder.java         |  52 ++-
 .../org/apache/metamodel/hbase/HBaseTable.java  | 119 +++--
 .../metamodel/hbase/HBaseTableDropBuilder.java  |  17 +-
 .../metamodel/hbase/HBaseUpdateCallback.java    |  26 +-
 .../apache/metamodel/hbase/CreateTableTest.java | 193 ++++++++
 .../apache/metamodel/hbase/DeleteRowTest.java   | 169 +++++++
 .../apache/metamodel/hbase/DropTableTest.java   |  87 ++++
 .../metamodel/hbase/HBaseDataContextTest.java   |  39 +-
 .../apache/metamodel/hbase/HBaseTestCase.java   |  57 +--
 .../hbase/HBaseUpdateCallbackTest.java          | 252 ++++++-----
 .../apache/metamodel/hbase/InsertRowTest.java   | 435 +++++++++++++++++++
 15 files changed, 1344 insertions(+), 390 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
index ed7b17c..979322e 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java
@@ -49,16 +49,22 @@ public final class HBaseClient {
     }
 
     /**
-     * Write a single row of values to a HBase table
-     * @param hBaseTable
+     * Insert a single row of values to a HBase table.
+     * @param tableName
      * @param columns
      * @param values
-     * @throws IOException
+     * @throws IllegalArgumentException when any parameter is null or the indexOfIdColumn is impossible
+     * @throws MetaModelException when no ID-column is found.
+     * @throws MetaModelException when a {@link IOException} is catched
      */
-    public void writeRow(HBaseTable hBaseTable, HBaseColumn[] columns, Object[] values) throws IOException {
-        try (final Table table = _connection.getTable(TableName.valueOf(hBaseTable.getName()))) {
-            int indexOfIdColumn = getIndexOfIdColumn(columns);
-
+    // TODO: Use the ColumnTypes to determine the inserts. Now the toString() method is called on the object.
+    public void insertRow(String tableName, HBaseColumn[] columns, Object[] values, int indexOfIdColumn) {
+        if (tableName == null || columns == null || values == null || indexOfIdColumn >= values.length
+                || values[indexOfIdColumn] == null) {
+            throw new IllegalArgumentException(
+                    "Can't insert a row without having (correct) tableName, columns, values or indexOfIdColumn");
+        }
+        try (final Table table = _connection.getTable(TableName.valueOf(tableName))) {
             // Create a put with the values of indexOfIdColumn as rowkey
             final Put put = new Put(Bytes.toBytes(values[indexOfIdColumn].toString()));
 
@@ -71,56 +77,42 @@ public final class HBaseClient {
             }
             // Add the put to the table
             table.put(put);
+        } catch (IOException e) {
+            throw new MetaModelException(e);
         }
     }
 
     /**
-     * Gets the index of the ID-column
-     * Throws an {@link MetaModelException} when no ID-column is found.
-     * @param columns
-     * @return 
-     */
-    private int getIndexOfIdColumn(HBaseColumn[] columns) {
-        int indexOfIdColumn = 0;
-        boolean idColumnFound = false;
-        while (!idColumnFound && indexOfIdColumn < columns.length) {
-            if (columns[indexOfIdColumn].getColumnFamily().equals(HBaseDataContext.FIELD_ID)) {
-                idColumnFound = true;
-            } else {
-                indexOfIdColumn++;
-            }
-        }
-        if (!idColumnFound) {
-            throw new MetaModelException("The ID Column family was not found");
-        }
-        return indexOfIdColumn;
-    }
-
-    /**
      * Delete 1 row based on the key
-     * @param hBaseTable
-     * @param key
-     * @throws IOException
+     * @param tableName
+     * @param rowKey
+     * @throws IllegalArgumentException when any parameter is null
+     * @throws MetaModelException when a {@link IOException} is catched
      */
-    public void deleteRow(HBaseTable hBaseTable, Object key) throws IOException {
-        try (final Table table = _connection.getTable(TableName.valueOf(hBaseTable.getName()));) {
-            if (rowExists(table, key) == true) {
-                table.delete(new Delete(Bytes.toBytes(key.toString())));
+    public void deleteRow(String tableName, Object rowKey) {
+        if (tableName == null || rowKey == null) {
+            throw new IllegalArgumentException("Can't delete a row without having tableName or rowKey");
+        }
+        try (final Table table = _connection.getTable(TableName.valueOf(tableName));) {
+            if (rowExists(table, rowKey) == true) {
+                table.delete(new Delete(Bytes.toBytes(rowKey.toString())));
             } else {
-                logger.warn("Rowkey with value " + key.toString() + " doesn't exist in the table");
+                logger.warn("Rowkey with value " + rowKey.toString() + " doesn't exist in the table");
             }
+        } catch (IOException e) {
+            throw new MetaModelException(e);
         }
     }
 
     /**
      * Checks in the HBase datastore if a row exists based on the key
      * @param table
-     * @param key
+     * @param rowKey
      * @return boolean
      * @throws IOException
      */
-    private boolean rowExists(Table table, Object key) throws IOException {
-        final Get get = new Get(Bytes.toBytes(key.toString()));
+    private boolean rowExists(Table table, Object rowKey) throws IOException {
+        final Get get = new Get(Bytes.toBytes(rowKey.toString()));
         return !table.get(get).isEmpty();
     }
 
@@ -128,9 +120,13 @@ public final class HBaseClient {
      * Creates a HBase table based on a tableName and it's columnFamilies
      * @param tableName
      * @param columnFamilies
-     * @throws IOException
+     * @throws IllegalArgumentException when any parameter is null
+     * @throws MetaModelException when a {@link IOException} is catched
      */
-    public void createTable(String tableName, Set<String> columnFamilies) throws IOException {
+    public void createTable(String tableName, Set<String> columnFamilies) {
+        if (tableName == null || columnFamilies == null || columnFamilies.isEmpty()) {
+            throw new IllegalArgumentException("Can't create a table without having the tableName or columnFamilies");
+        }
         try (final Admin admin = _connection.getAdmin()) {
             final TableName hBasetableName = TableName.valueOf(tableName);
             final HTableDescriptor tableDescriptor = new HTableDescriptor(hBasetableName);
@@ -142,19 +138,27 @@ public final class HBaseClient {
                 }
             }
             admin.createTable(tableDescriptor);
+        } catch (IOException e) {
+            throw new MetaModelException(e);
         }
     }
 
     /**
      * Disable and drop a table from a HBase datastore
      * @param tableName
-     * @throws IOException
+     * @throws IllegalArgumentException when tableName is null
+     * @throws MetaModelException when a {@link IOException} is catched
      */
-    public void dropTable(String tableName) throws IOException {
+    public void dropTable(String tableName) {
+        if (tableName == null) {
+            throw new IllegalArgumentException("Can't drop a table without having the tableName");
+        }
         try (final Admin admin = _connection.getAdmin()) {
             final TableName hBasetableName = TableName.valueOf(tableName);
             admin.disableTable(hBasetableName); // A table must be disabled first, before it can be deleted
             admin.deleteTable(hBasetableName);
+        } catch (IOException e) {
+            throw new MetaModelException(e);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
index bd57c1c..d6554c4 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java
@@ -18,22 +18,27 @@
  */
 package org.apache.metamodel.hbase;
 
+import java.util.Arrays;
+import java.util.Iterator;
 import java.util.LinkedHashSet;
+import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
 
-import org.apache.metamodel.schema.AbstractColumn;
+import org.apache.metamodel.schema.Column;
 import org.apache.metamodel.schema.ColumnType;
 import org.apache.metamodel.schema.ColumnTypeImpl;
+import org.apache.metamodel.schema.MutableColumn;
 import org.apache.metamodel.schema.SuperColumnType;
 import org.apache.metamodel.schema.Table;
 
-public final class HBaseColumn extends AbstractColumn {
+public final class HBaseColumn extends MutableColumn {
+    public final static ColumnType DEFAULT_COLUMN_TYPE_FOR_ID_COLUMN = new ColumnTypeImpl("BYTE[]",
+            SuperColumnType.LITERAL_TYPE);
+    public final static ColumnType DEFAULT_COLUMN_TYPE_FOR_COLUMN_FAMILIES = ColumnType.LIST;
+
     private final String columnFamily;
     private final String qualifier;
-    private final Table table;
-    private final boolean primaryKey;
-    private final ColumnType columnType;
-    private final int columnNumber;
 
     public HBaseColumn(final String columnFamily, final Table table) {
         this(columnFamily, null, table, -1);
@@ -48,23 +53,32 @@ public final class HBaseColumn extends AbstractColumn {
     }
 
     public HBaseColumn(final String columnFamily, final String qualifier, final Table table, final int columnNumber) {
+        this(columnFamily, qualifier, table, columnNumber, null);
+    }
+
+    public HBaseColumn(final String columnFamily, final String qualifier, final Table table, final int columnNumber,
+            final ColumnType columnType) {
+        super(columnFamily, table);
         if (columnFamily == null) {
             throw new IllegalArgumentException("Column family isn't allowed to be null.");
-        } else if (table == null) {
-            throw new IllegalArgumentException("Table isn't allowed to be null.");
+        } else if (table == null || !(table instanceof HBaseTable)) {
+            throw new IllegalArgumentException("Table is null or isn't a HBaseTable.");
         }
 
         this.columnFamily = columnFamily;
         this.qualifier = qualifier;
-        this.table = table;
-        this.columnNumber = columnNumber;
+        setColumnNumber(columnNumber);
+        setPrimaryKey(HBaseDataContext.FIELD_ID.equals(columnFamily));
 
-        primaryKey = HBaseDataContext.FIELD_ID.equals(columnFamily);
-
-        if (primaryKey || qualifier != null) {
-            columnType = new ColumnTypeImpl("BYTE[]", SuperColumnType.LITERAL_TYPE);
+        // Set the columnType
+        if (columnType != null) {
+            setType(columnType);
         } else {
-            columnType = ColumnType.LIST;
+            if (isPrimaryKey() || qualifier != null) {
+                setType(DEFAULT_COLUMN_TYPE_FOR_ID_COLUMN);
+            } else {
+                setType(DEFAULT_COLUMN_TYPE_FOR_COLUMN_FAMILIES);
+            }
         }
     }
 
@@ -85,23 +99,8 @@ public final class HBaseColumn extends AbstractColumn {
     }
 
     @Override
-    public int getColumnNumber() {
-        return columnNumber;
-    }
-
-    @Override
-    public ColumnType getType() {
-        return columnType;
-    }
-
-    @Override
-    public Table getTable() {
-        return table;
-    }
-
-    @Override
     public Boolean isNullable() {
-        return !primaryKey;
+        return !isPrimaryKey();
     }
 
     @Override
@@ -126,25 +125,107 @@ public final class HBaseColumn extends AbstractColumn {
     }
 
     @Override
-    public boolean isPrimaryKey() {
-        return primaryKey;
-    }
-
-    @Override
     public String getQuote() {
         return null;
     }
 
     /**
-     * Creates a set of columnFamilies out of an array of hbaseColumns
-     * @param hbaseColumns
+     * Creates a set of columnFamilies out of a list of hbaseColumns
+     * @param columns
      * @return {@link LinkedHashSet}
      */
-    public static Set<String> getColumnFamilies(HBaseColumn[] hbaseColumns) {
-        final LinkedHashSet<String> columnFamilies = new LinkedHashSet<String>();
-        for (int i = 0; i < hbaseColumns.length; i++) {
-            columnFamilies.add(hbaseColumns[i].getColumnFamily());
+    public static Set<String> getColumnFamilies(List<HBaseColumn> columns) {
+        final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
+        for (HBaseColumn column : columns) {
+            columnFamilies.add(column.getColumnFamily());
         }
         return columnFamilies;
     }
+
+    /**
+     * Returns the index of the ID-column (see {@link HBaseDataContext#FIELD_ID}) in an array of HBaseColumns.
+     * When no ID-column is found, then null is returned.
+     * @param columns
+     * @return {@link Integer}
+     */
+    public static Integer findIndexOfIdColumn(List<HBaseColumn> columns) {
+        int i = 0;
+        Integer indexOfIDColumn = null;
+        Iterator<HBaseColumn> iterator = columns.iterator();
+        while (indexOfIDColumn == null && iterator.hasNext()) {
+            indexOfIDColumn = findIndexOfIdColumn(iterator.next().getColumnFamily(), i);
+            if (indexOfIDColumn == null) {
+                i++;
+            }
+        }
+        return indexOfIDColumn;
+    }
+
+    /**
+     * Returns the index of the ID-column (see {@link HBaseDataContext#FIELD_ID}) in an array of columnNames.
+     * When no ID-column is found, then null is returned.
+     * @param columnNames
+     * @return {@link Integer}
+     */
+    public static Integer findIndexOfIdColumn(String[] columnNames) {
+        int i = 0;
+        Integer indexOfIDColumn = null;
+        while (indexOfIDColumn == null && i < columnNames.length) {
+            indexOfIDColumn = findIndexOfIdColumn(columnNames[i], i);
+            if (indexOfIDColumn == null) {
+                i++;
+            }
+        }
+        return indexOfIDColumn;
+    }
+
+    /**
+     * Returns the index of the ID-column (see {@link HBaseDataContext#FIELD_ID})
+     * When no ID-column is found, then null is returned.
+     * @param columnNames
+     * @return {@link Integer}
+     */
+    private static Integer findIndexOfIdColumn(String columnName, int index) {
+        Integer indexOfIDColumn = null;
+        if (columnName.equals(HBaseDataContext.FIELD_ID)) {
+            indexOfIDColumn = new Integer(index);
+        }
+        return indexOfIDColumn;
+    }
+
+    /**
+     * Converts a list of {@link Column}'s to a list of {@link HBaseColumn}'s
+     * @param columns
+     * @return {@link List}<{@link HBaseColumn}>
+     */
+    public static List<HBaseColumn> convertToHBaseColumnsList(List<Column> columns) {
+        return columns.stream().map(column -> (HBaseColumn) column).collect(Collectors.toList());
+    }
+
+    /**
+     * Converts a list of {@link HBaseColumn}'s to a list of {@link Column}'s
+     * @param columns
+     * @return {@link List}<{@link Column}>
+     */
+    public static List<Column> convertToColumnsList(List<HBaseColumn> columns) {
+        return columns.stream().map(column -> (Column) column).collect(Collectors.toList());
+    }
+
+    /**
+     * Converts a list of {@link HBaseColumn}'s to an array of {@link HBaseColumn}'s
+     * @param columns
+     * @return Array of {@link HBaseColumn}
+     */
+    public static HBaseColumn[] convertToHBaseColumnsArray(List<HBaseColumn> columns) {
+        return columns.stream().map(column -> column).toArray(size -> new HBaseColumn[size]);
+    }
+
+    /**
+     * Converts a array of {@link Column}'s to an array of {@link HBaseColumn}'s
+     * @param columns
+     * @return Array of {@link HBaseColumn}
+     */
+    public static HBaseColumn[] convertToHBaseColumnsArray(Column[] columns) {
+        return Arrays.stream(columns).map(column -> (HBaseColumn) column).toArray(size -> new HBaseColumn[size]);
+    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
index ffdc20a..138e32b 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java
@@ -18,7 +18,6 @@
  */
 package org.apache.metamodel.hbase;
 
-import java.io.IOException;
 import java.util.Iterator;
 import java.util.Set;
 
@@ -64,12 +63,7 @@ public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseU
         final Table table = getTable();
 
         // Add the table to the datastore
-        try {
-            final HBaseClient hBaseClient = getUpdateCallback().getHBaseClient();
-            hBaseClient.createTable(table.getName(), _columnFamilies);
-        } catch (IOException e) {
-            throw new MetaModelException(e);
-        }
+        getUpdateCallback().getHBaseClient().createTable(table.getName(), _columnFamilies);
 
         // Update the schema
         addNewTableToSchema(table);
@@ -108,7 +102,7 @@ public class HBaseCreateTableBuilder extends AbstractTableCreationBuilder<HBaseU
     /**
      * Add the new {@link Table} to the {@link MutableSchema}
      * @param table
-     * @param updateCallback
+     * @param data.updateCallback
      * @return {@link MutableSchema}
      */
     private void addNewTableToSchema(final Table table) {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
index 3ba4ef8..99d2571 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowDeletionBuilder.java
@@ -18,38 +18,41 @@
  */
 package org.apache.metamodel.hbase;
 
-import java.io.IOException;
-
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.delete.AbstractRowDeletionBuilder;
 import org.apache.metamodel.schema.Table;
 
 /**
- * A builder-class to delete rows in a HBase datastore
+ * A builder-class to delete rows based on their keys in a HBase datastore
  */
 public class HBaseRowDeletionBuilder extends AbstractRowDeletionBuilder {
 
     private HBaseClient _hBaseClient;
     private Object _key;
 
+    /**
+     * Creates a {@link HBaseRowDeletionBuilder}
+     * @param hBaseWriter
+     * @param table
+     * @throws IllegalArgumentException when the hBaseWriter is null
+     */
     public HBaseRowDeletionBuilder(final HBaseClient hBaseWriter, final Table table) {
         super(table);
         if (hBaseWriter == null) {
-            throw new IllegalArgumentException("UpdateCallback cannot be null");
+            throw new IllegalArgumentException("hBaseClient cannot be null");
         }
         this._hBaseClient = hBaseWriter;
     }
 
+    /** 
+     * @throws MetaModelException when value is null
+     */
     @Override
     public synchronized void execute() {
         if (_key == null) {
             throw new MetaModelException("Key cannot be null");
         }
-        try {
-            _hBaseClient.deleteRow((HBaseTable) getTable(), _key);
-        } catch (IOException e) {
-            throw new MetaModelException(e);
-        }
+        _hBaseClient.deleteRow(getTable().getName(), _key);
     }
 
     public void setKey(Object _key) {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
index 7272f76..9e36a4d 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java
@@ -18,8 +18,6 @@
  */
 package org.apache.metamodel.hbase;
 
-import java.io.IOException;
-import java.util.Arrays;
 import java.util.List;
 
 import org.apache.metamodel.MetaModelException;
@@ -27,20 +25,44 @@ import org.apache.metamodel.insert.AbstractRowInsertionBuilder;
 import org.apache.metamodel.schema.Column;
 
 /**
- * A builder-class to insert rows in a HBase datastore
+ * A builder-class to insert rows in a HBase datastore.
  */
+// TODO: Possible future improvement: Make it possible to change the columns for each execute.
+// Now each row will get exactly the same columns.
 public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
+    final Integer _indexOfIdColumn;
+
+    /**
+     * Creates a {@link HBaseRowInsertionBuilder}. The table and the column's columnFamilies are checked to exist in the schema.
+     * @param updateCallback
+     * @param table
+     * @param columns
+     * @throws IllegalArgumentException the columns list can't be null or empty
+     * @throws MetaModelException when no ID-column is found.
+     */
     public HBaseRowInsertionBuilder(final HBaseUpdateCallback updateCallback, final HBaseTable table,
             final List<Column> columns) {
         super(updateCallback, table, columns);
+        if (columns.isEmpty()) { // TODO: Columns null will already result in a NullPointer at the super. Should the
+                                 // super get a extra check?
+            throw new IllegalArgumentException("The hbaseColumns list is null or empty");
+        }
+
+        this._indexOfIdColumn = HBaseColumn.findIndexOfIdColumn(HBaseColumn.convertToHBaseColumnsList(columns));
+        if (_indexOfIdColumn == null) {
+            throw new MetaModelException("The ID-Column was not found");
+        }
+
         checkTable(updateCallback, table);
+        table.checkForNotMatchingColumnFamilies(HBaseColumn.getColumnFamilies(HBaseColumn.convertToHBaseColumnsList(
+                columns)));
     }
 
     /**
-     * Check if the table exits and it's columnFamilies exist
-     * If the table doesn't exist, then a {@link MetaModelException} is thrown
+     * Check if the table and it's columnFamilies exist in the schema
      * @param updateCallback
      * @param tableGettingInserts
+     * @throws MetaModelException If the table or the columnFamilies don't exist
      */
     private void checkTable(final HBaseUpdateCallback updateCallback, final HBaseTable tableGettingInserts) {
         final HBaseTable tableInSchema = (HBaseTable) updateCallback.getDataContext().getDefaultSchema().getTableByName(
@@ -49,28 +71,18 @@ public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseU
             throw new MetaModelException("Trying to insert data into table: " + tableGettingInserts.getName()
                     + ", which doesn't exist yet");
         }
-        tableInSchema.checkForNotMatchingColumns(tableGettingInserts.getColumnNames());
+        tableInSchema.checkForNotMatchingColumnFamilies(HBaseColumn.getColumnFamilies(tableGettingInserts
+                .getHBaseColumnsInternal()));
     }
 
     @Override
     public synchronized void execute() {
-        if (getColumns() == null || getColumns().length == 0) {
-            throw new MetaModelException("The hbaseColumns-array is null or empty");
-        }
-        if (getValues() == null || getValues().length == 0) {
-            throw new MetaModelException("The values-array is null or empty");
-        }
-        try {
-            final HBaseClient hBaseClient = getUpdateCallback().getHBaseClient();
-            hBaseClient.writeRow((HBaseTable) getTable(), getColumns(), getValues());
-        } catch (IOException e) {
-            throw new MetaModelException(e);
-        }
+        getUpdateCallback().getHBaseClient().insertRow(getTable().getName(), getColumns(), getValues(), _indexOfIdColumn
+                .intValue());
     }
 
     @Override
     public HBaseColumn[] getColumns() {
-        return Arrays.stream(super.getColumns()).map(column -> (HBaseColumn) column).toArray(
-                size -> new HBaseColumn[size]);
+        return HBaseColumn.convertToHBaseColumnsArray(super.getColumns());
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
index fee2f5d..9f482ac 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
@@ -18,13 +18,14 @@
  */
 package org.apache.metamodel.hbase;
 
+import java.util.Iterator;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.schema.Column;
 import org.apache.metamodel.schema.ColumnType;
-import org.apache.metamodel.schema.MutableColumn;
 import org.apache.metamodel.schema.MutableSchema;
 import org.apache.metamodel.schema.MutableTable;
 import org.apache.metamodel.schema.TableType;
@@ -44,8 +45,8 @@ final class HBaseTable extends MutableTable {
     private final transient ColumnType _defaultRowKeyColumnType;
 
     /**
-     * Creates an HBaseTable. If the tableDef variable doesn't include the ID column (see {@link HBaseDataContext#FIELD_ID}). 
-     * Then it's first inserted.
+     * Creates an HBaseTable. If the tableDef variable doesn't include the ID-column (see {@link HBaseDataContext#FIELD_ID}). 
+     * Then it's first added.
      * @param dataContext
      * @param tableDef Table definition. The tableName, columnNames and columnTypes variables are used.
      * @param schema {@link MutableSchema} where the table belongs to.
@@ -57,7 +58,15 @@ final class HBaseTable extends MutableTable {
         super(tableDef.getName(), TableType.TABLE, schema);
         _dataContext = dataContext;
         _defaultRowKeyColumnType = defaultRowKeyColumnType;
+        addColumns(tableDef, defaultRowKeyColumnType);
+    }
 
+    /**
+     * Add multiple columns to this table
+     * @param tableDef
+     * @param defaultRowKeyColumnType
+     */
+    private void addColumns(SimpleTableDef tableDef, ColumnType defaultRowKeyColumnType) {
         // Add the columns
         final String[] columnNames = tableDef.getColumnNames();
         if (columnNames == null || columnNames.length == 0) {
@@ -66,79 +75,55 @@ final class HBaseTable extends MutableTable {
             final ColumnType[] columnTypes = tableDef.getColumnTypes();
 
             // Find the ID-Column
-            boolean idColumnFound = false;
-            int indexOfIDColumn = 0;
-            while (!idColumnFound && indexOfIDColumn < columnNames.length) {
-                if (columnNames[indexOfIDColumn].equals(HBaseDataContext.FIELD_ID)) {
-                    idColumnFound = true;
-                } else {
-                    indexOfIDColumn++;
-                }
-            }
+            Integer indexOfIDColumn = HBaseColumn.findIndexOfIdColumn(columnNames);
+            boolean idColumnFound = indexOfIDColumn != null;
 
-            int columnNumber = indexOfIDColumn + 1; // ColumnNumbers start from 1
-
-            // Add the ID-Column, even if the column wasn't included in columnNames
-            ColumnType columnType;
+            // ColumnNumbers start from 1
             if (idColumnFound) {
-                columnType = columnTypes[indexOfIDColumn];
+                addColumn(HBaseDataContext.FIELD_ID, columnTypes[indexOfIDColumn.intValue()], indexOfIDColumn.intValue()
+                        + 1);
             } else {
-                columnType = defaultRowKeyColumnType;
+                addColumn(HBaseDataContext.FIELD_ID, defaultRowKeyColumnType, 1);
             }
-            final MutableColumn idColumn = new MutableColumn(HBaseDataContext.FIELD_ID, columnType)
-                    .setPrimaryKey(true)
-                    .setColumnNumber(columnNumber)
-                    .setTable(this);
-            addColumn(idColumn);
 
             // Add the other columns
             for (int i = 0; i < columnNames.length; i++) {
-                final String columnName = columnNames[i];
-                if (idColumnFound) {
-                    columnNumber = i + 1; // ColumnNumbers start from 1
-                } else {
-                    columnNumber = i + 2; // ColumnNumbers start from 1 + the ID-column has just been created
-                }
-                if (!HBaseDataContext.FIELD_ID.equals(columnName)) {
-                    final ColumnType type = columnTypes[i];
-                    final MutableColumn column = new MutableColumn(columnName, type);
-                    column.setTable(this);
-                    column.setColumnNumber(columnNumber);
-                    addColumn(column);
-                    columnNumber++;
+                if (!HBaseDataContext.FIELD_ID.equals(columnNames[i])) {
+                    if (idColumnFound) {
+                        addColumn(columnNames[i], columnTypes[i], i + 1);
+                    } else {
+                        addColumn(columnNames[i], columnTypes[i], i + 2);
+                    }
                 }
             }
         }
     }
 
+    /**
+     * Add a column to this table
+     * @param columnName
+     * @param columnType
+     * @param columnNumber
+     */
+    private void addColumn(final String columnName, final ColumnType columnType, final int columnNumber) {
+        addColumn(new HBaseColumn(columnName, null, this, columnNumber, columnType));
+    }
+
     @Override
     protected synchronized List<Column> getColumnsInternal() {
         final List<Column> columnsInternal = super.getColumnsInternal();
         if (columnsInternal.isEmpty() && _dataContext != null) {
-            try {
-                final org.apache.hadoop.hbase.client.Table table = _dataContext.getHTable(getName());
-                int columnNumber = 1;
-
-                final MutableColumn idColumn = new MutableColumn(HBaseDataContext.FIELD_ID, _defaultRowKeyColumnType)
-                        .setPrimaryKey(true)
-                        .setColumnNumber(columnNumber)
-                        .setTable(this);
-                addColumn(idColumn);
-                columnNumber++;
+            try (final org.apache.hadoop.hbase.client.Table table = _dataContext.getHTable(getName())) {
+                // Add the ID-Column (with columnNumber = 1)
+                addColumn(HBaseDataContext.FIELD_ID, _defaultRowKeyColumnType, 1);
 
                 // What about timestamp?
 
+                // Add the other column (with columnNumbers starting from 2)
                 final HColumnDescriptor[] columnFamilies = table.getTableDescriptor().getColumnFamilies();
                 for (int i = 0; i < columnFamilies.length; i++) {
-                    final HColumnDescriptor columnDescriptor = columnFamilies[i];
-                    final String columnFamilyName = columnDescriptor.getNameAsString();
-                    // HBase column families are always unstructured maps.
-                    final ColumnType type = ColumnType.MAP;
-                    final MutableColumn column = new MutableColumn(columnFamilyName, type);
-                    column.setTable(this);
-                    column.setColumnNumber(columnNumber);
-                    columnNumber++;
-                    addColumn(column);
+                    addColumn(columnFamilies[i].getNameAsString(), HBaseColumn.DEFAULT_COLUMN_TYPE_FOR_COLUMN_FAMILIES,
+                            i + 2);
                 }
             } catch (Exception e) {
                 throw new MetaModelException("Could not resolve table ", e);
@@ -149,19 +134,18 @@ final class HBaseTable extends MutableTable {
 
     /**
      * Check if a list of columnNames all exist in this table
-     * If a column doesn't exist, then a {@link MetaModelException} is thrown
      * @param columnNamesOfCheckedTable
+     * @throws MetaModelException If a column doesn't exist
      */
-    public void checkForNotMatchingColumns(final List<String> columnNamesOfCheckedTable) {
-        final List<String> columnsNamesOfExistingTable = getColumnNames();
+    public void checkForNotMatchingColumnFamilies(final Set<String> columnNamesOfCheckedTable) {
+        Set<String> columnFamilyNamesOfExistingTable = HBaseColumn.getColumnFamilies(getHBaseColumnsInternal());
+
         for (String columnNameOfCheckedTable : columnNamesOfCheckedTable) {
             boolean matchingColumnFound = false;
-            int i = 0;
-            while (!matchingColumnFound && i < columnsNamesOfExistingTable.size()) {
-                if (columnNameOfCheckedTable.equals(columnsNamesOfExistingTable.get(i))) {
+            Iterator<String> iterator = columnFamilyNamesOfExistingTable.iterator();
+            while (!matchingColumnFound && iterator.hasNext()) {
+                if (columnNameOfCheckedTable.equals(iterator.next())) {
                     matchingColumnFound = true;
-                } else {
-                    i++;
                 }
             }
             if (!matchingColumnFound) {
@@ -170,4 +154,13 @@ final class HBaseTable extends MutableTable {
             }
         }
     }
+
+    /**
+     * Returns a list of {@link HBaseColumn}'s from {@link HBaseTable#getColumnsInternal()}, 
+     * which returns a list of {@link Column}'s
+     * @return {@link List}<{@link HBaseColumn}>
+     */
+    public List<HBaseColumn> getHBaseColumnsInternal() {
+        return HBaseColumn.convertToHBaseColumnsList(getColumnsInternal());
+    }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
index 374e325..a08ab83 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTableDropBuilder.java
@@ -18,8 +18,6 @@
  */
 package org.apache.metamodel.hbase;
 
-import java.io.IOException;
-
 import org.apache.metamodel.MetaModelException;
 import org.apache.metamodel.drop.AbstractTableDropBuilder;
 import org.apache.metamodel.schema.MutableSchema;
@@ -41,16 +39,11 @@ public class HBaseTableDropBuilder extends AbstractTableDropBuilder {
 
     @Override
     public void execute() {
-        try {
-            // Remove from the datastore
-            final HBaseClient hBaseClient = _updateCallback.getHBaseClient();
-            final Table table = getTable();
-            hBaseClient.dropTable(table.getName());
+        // Remove from the datastore
+        final Table table = getTable();
+        _updateCallback.getHBaseClient().dropTable(table.getName());
 
-            // Remove from schema
-            ((MutableSchema) table.getSchema()).removeTable(table);
-        } catch (IOException e) {
-            throw new MetaModelException(e);
-        }
+        // Remove from schema
+        ((MutableSchema) table.getSchema()).removeTable(table);
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
index 6d35508..b154275 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java
@@ -20,7 +20,6 @@ package org.apache.metamodel.hbase;
 
 import java.util.List;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 import org.apache.metamodel.AbstractUpdateCallback;
 import org.apache.metamodel.UpdateCallback;
@@ -28,7 +27,6 @@ import org.apache.metamodel.create.TableCreationBuilder;
 import org.apache.metamodel.delete.RowDeletionBuilder;
 import org.apache.metamodel.drop.TableDropBuilder;
 import org.apache.metamodel.insert.RowInsertionBuilder;
-import org.apache.metamodel.schema.Column;
 import org.apache.metamodel.schema.Schema;
 import org.apache.metamodel.schema.Table;
 
@@ -70,19 +68,28 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
         return new HBaseTableDropBuilder(table, this);
     }
 
+    /**
+     * @throws IllegalArgumentException when table isn't a {@link HBaseTable}
+     */
     @Override
     public RowInsertionBuilder insertInto(final Table table) {
         throw new UnsupportedOperationException(
                 "We need an explicit list of columns when inserting into an HBase table.");
     }
 
-    public RowInsertionBuilder insertInto(final Table table, final List<HBaseColumn> columns) {
+    /**
+     * Initiates the building of a row insertion operation. 
+     * @param table Table to get inserts.
+     * @param columns List of {@link HBaseColumn} to insert on.
+     * @return {@link HBaseRowInsertionBuilder}
+     * @throws IllegalArgumentException The table must be an {@link HBaseTable} and the columns list can't be null or empty
+     */
+    public HBaseRowInsertionBuilder insertInto(final Table table, final List<HBaseColumn> columns) {
+        if (columns == null || columns.isEmpty()) {
+            throw new IllegalArgumentException("The hbaseColumns list is null or empty");
+        }
         if (table instanceof HBaseTable) {
-            return new HBaseRowInsertionBuilder(this, (HBaseTable) table, columns
-                    .stream()
-                    .map(obj -> (Column) obj)
-                    .collect(
-                    Collectors.toList()));
+            return new HBaseRowInsertionBuilder(this, (HBaseTable) table, HBaseColumn.convertToColumnsList(columns));
         } else {
             throw new IllegalArgumentException("Not an HBase table: " + table);
         }
@@ -93,6 +100,9 @@ public class HBaseUpdateCallback extends AbstractUpdateCallback implements Updat
         return true;
     }
 
+    /**
+     * @throws IllegalArgumentException when table isn't a {@link HBaseTable}
+     */
     @Override
     public RowDeletionBuilder deleteFrom(Table table) {
         if (table instanceof HBaseTable) {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
new file mode 100644
index 0000000..e804a67
--- /dev/null
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/CreateTableTest.java
@@ -0,0 +1,193 @@
+/**
+ * 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.metamodel.hbase;
+
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import org.apache.metamodel.MetaModelException;
+import org.apache.metamodel.schema.ImmutableSchema;
+
+public class CreateTableTest extends HBaseUpdateCallbackTest {
+
+    /**
+     * Check if creating table is supported
+     */
+    public void testDropTableSupported() {
+        assertTrue(getUpdateCallback().isCreateTableSupported());
+    }
+
+    /**
+     * Create a table with an immutableSchema, should throw a IllegalArgumentException
+     */
+    public void testWrongSchema() {
+        final ImmutableSchema immutableSchema = new ImmutableSchema(getSchema());
+        try {
+            getUpdateCallback().createTable(immutableSchema, TABLE_NAME).execute();
+            fail("Should get an exception that the schema isn't mutable");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Not a mutable schema: " + immutableSchema, e.getMessage());
+        }
+    }
+
+    /**
+     * Create a table without columnFamilies, should throw a MetaModelException
+     */
+    public void testCreateTableWithoutColumnFamilies() {
+        try {
+            getUpdateCallback().createTable(getSchema(), TABLE_NAME).execute();
+            fail("Should get an exception that the columnFamilies haven't been set");
+        } catch (MetaModelException e) {
+            assertEquals("Creating a table without columnFamilies", e.getMessage());
+        }
+    }
+
+    /**
+     * Create a table with columnFamilies null, should throw a MetaModelException
+     */
+    public void testColumnFamiliesNull() {
+        try {
+            getUpdateCallback().createTable(getSchema(), TABLE_NAME, null).execute();
+            fail("Should get an exception that the columnFamilies haven't been set");
+        } catch (MetaModelException e) {
+            assertEquals("Creating a table without columnFamilies", e.getMessage());
+        }
+    }
+
+    /**
+     * Create a table with columnFamilies empty, should throw a MetaModelException
+     */
+    public void testColumnFamiliesEmpty() {
+        try {
+            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<String>();
+            getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
+            fail("Should get an exception that the columnFamilies haven't been set");
+        } catch (MetaModelException e) {
+            assertEquals("Creating a table without columnFamilies", e.getMessage());
+        }
+    }
+
+    /**
+     * Create a table without the ID-Column, should throw a MetaModelException
+     */
+    public void testCreateTableWithoutIDColumn() {
+        if (isConfigured()) {
+            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, null, CF_FOO, CF_BAR);
+            final Set<String> columnFamilies = HBaseColumn.getColumnFamilies(getHBaseColumnsFromMap(row));
+            try {
+                final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
+                        .createTable(getSchema(), TABLE_NAME);
+
+                hBaseCreateTableBuilder.setColumnFamilies(columnFamilies);
+                hBaseCreateTableBuilder.execute();
+                fail("Should get an exception that the ID-colum is missing");
+            } catch (MetaModelException e) {
+                assertEquals("ColumnFamily: " + HBaseDataContext.FIELD_ID + " not found", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Creating a HBaseClient with the tableName null, should throw a exception
+     */
+    public void testCreatingTheHBaseClientWithTableNameNull() {
+        try {
+            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
+            columnFamilies.add("1");
+            new HBaseClient(getDataContext().getConnection()).createTable(null, columnFamilies);
+            fail("Should get an exception that tableName is null");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
+        }
+    }
+
+    /**
+     * Creating a HBaseClient with the tableName null, should throw a exception
+     */
+    public void testCreatingTheHBaseClientWithColumnFamiliesNull() {
+        try {
+            new HBaseClient(getDataContext().getConnection()).createTable("1", null);
+            fail("Should get an exception that columnFamilies is null");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
+        }
+    }
+
+    /**
+     * Creating a HBaseClient with the tableName null, should throw a exception
+     */
+    public void testCreatingTheHBaseClientWithColumnFamiliesEmpty() {
+        try {
+            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
+            new HBaseClient(getDataContext().getConnection()).createTable("1", columnFamilies);
+            fail("Should get an exception that columnFamilies is empty");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't create a table without having the tableName or columnFamilies", e.getMessage());
+        }
+    }
+
+    /**
+     * Goodflow. Create a table including the ID-Column (columnFamilies not in constructor), should work
+     */
+    public void testSettingColumnFamiliesAfterConstrutor() {
+        if (isConfigured()) {
+            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+            final Set<String> columnFamilies = HBaseColumn.getColumnFamilies(getHBaseColumnsFromMap(row));
+            try {
+                final HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) getUpdateCallback()
+                        .createTable(getSchema(), TABLE_NAME);
+
+                hBaseCreateTableBuilder.setColumnFamilies(columnFamilies);
+                hBaseCreateTableBuilder.execute();
+                checkSuccesfullyInsertedTable();
+            } catch (Exception e) {
+                fail("Should not get an exception");
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Goodflow. Create a table including the ID-Column (columnFamilies in constructor), should work
+     */
+    public void testCreateTableColumnFamiliesInConstrutor() {
+        if (isConfigured()) {
+            final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+            final LinkedHashMap<HBaseColumn, Object> row = createRow(table, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR);
+            final Set<String> columnFamilies = HBaseColumn.getColumnFamilies(getHBaseColumnsFromMap(row));
+            try {
+                getUpdateCallback().createTable(getSchema(), TABLE_NAME, columnFamilies).execute();
+                checkSuccesfullyInsertedTable();
+            } catch (Exception e) {
+                fail("Should not get an exception");
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
new file mode 100644
index 0000000..8d21194
--- /dev/null
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DeleteRowTest.java
@@ -0,0 +1,169 @@
+/**
+ * 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.metamodel.hbase;
+
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.List;
+
+import org.apache.metamodel.MetaModelException;
+import org.apache.metamodel.schema.MutableTable;
+
+public class DeleteRowTest extends HBaseUpdateCallbackTest {
+
+    /**
+     * Delete is supported
+     */
+    public void testDeleteSupported() {
+        assertTrue(getUpdateCallback().isDeleteSupported());
+    }
+
+    /**
+     * Having the table type wrong, should throw an exception
+     */
+    public void testTableWrongType() {
+        final MutableTable mutableTable = new MutableTable();
+        try {
+            getUpdateCallback().deleteFrom(mutableTable);
+            fail("Should get an exception that the type of the table is wrong.");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Not an HBase table: " + mutableTable, e.getMessage());
+        }
+    }
+
+    /**
+     * Creating a HBaseRowDeletionBuilder with the hBaseClient null, should throw an exception
+     * @throws IOException 
+     */
+    public void testHBaseClientNullAtBuilder() throws IOException {
+        if (isConfigured()) {
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                new HBaseRowDeletionBuilder(null, existingTable);
+                fail("Should get an exception that hBaseClient can't be null.");
+            } catch (IllegalArgumentException e) {
+                assertEquals("hBaseClient cannot be null", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Not setting the rowkey, should throw an exception
+     * @throws IOException 
+     */
+    public void testNotSettingRowkey() throws IOException {
+        if (isConfigured()) {
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                getUpdateCallback().deleteFrom(existingTable).execute();
+                fail("Should get an exception that the columnFamily doesn't exist.");
+            } catch (MetaModelException e) {
+                assertEquals("Key cannot be null", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Creating a HBaseClient with the tableName null, should throw a exception
+     */
+    public void testCreatingTheHBaseClientWithTableNameNull() {
+        try {
+            new HBaseClient(getDataContext().getConnection()).deleteRow(null, new String("1"));
+            fail("Should get an exception that tableName is null");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't delete a row without having tableName or rowKey", e.getMessage());
+        }
+    }
+
+    /**
+     * Creating a HBaseClient with the rowKey null, should throw a exception
+     */
+    public void testCreatingTheHBaseClientWithRowKeyNull() {
+        try {
+            new HBaseClient(getDataContext().getConnection()).deleteRow("tableName", null);
+            fail("Should get an exception that rowKey is null");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't delete a row without having tableName or rowKey", e.getMessage());
+        }
+    }
+
+    /**
+     * Goodflow. Deleting a row, that doesn't exist, should not throw an exception
+     */
+    public void testDeletingNotExistingRow() {
+        if (isConfigured()) {
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+
+                checkRows(false);
+                final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback()
+                        .deleteFrom(existingTable);
+                rowDeletionBuilder.setKey(RK_1);
+                rowDeletionBuilder.execute();
+                checkRows(false);
+            } catch (Exception e) {
+                fail("Should not get an exception that the row doesn't exist.");
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Goodflow. Deleting a row succesfully.
+     */
+    public void testDeleteRowSuccesfully() {
+        if (isConfigured()) {
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                final LinkedHashMap<HBaseColumn, Object> row = createRow(existingTable, HBaseDataContext.FIELD_ID,
+                        CF_FOO, CF_BAR);
+                final List<HBaseColumn> columns = getHBaseColumnsFromMap(row);
+
+                checkRows(false);
+                final HBaseRowInsertionBuilder rowInsertionBuilder = getUpdateCallback().insertInto(existingTable,
+                        columns);
+                setValuesInInsertionBuilder(row, rowInsertionBuilder);
+                rowInsertionBuilder.execute();
+                checkRows(true);
+                final HBaseRowDeletionBuilder rowDeletionBuilder = (HBaseRowDeletionBuilder) getUpdateCallback()
+                        .deleteFrom(existingTable);
+                rowDeletionBuilder.setKey(RK_1);
+                rowDeletionBuilder.execute();
+                checkRows(false);
+            } catch (Exception e) {
+                fail("Should not get an exception on deleting a row.");
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
new file mode 100644
index 0000000..d5b19ef
--- /dev/null
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/DropTableTest.java
@@ -0,0 +1,87 @@
+/**
+ * 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.metamodel.hbase;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.metamodel.MetaModelException;
+
+public class DropTableTest extends HBaseUpdateCallbackTest {
+
+    /**
+     * Check if drop table is supported
+     */
+    public void testDropTableSupported() {
+        assertTrue(getUpdateCallback().isDropTableSupported());
+    }
+
+    /**
+     * Trying to drop a table, that doesn't exist in the datastore, should throw a exception
+     */
+    public void testDropTableThatDoesntExist() {
+        if (isConfigured()) {
+            try {
+                final HBaseTable table = createHBaseTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO, CF_BAR, null);
+                getUpdateCallback().dropTable(table).execute();
+                fail("Should get an exception that the table doesn't exist in the datastore");
+            } catch (MetaModelException e) {
+                assertEquals("Trying to delete a table that doesn't exist in the datastore.", e.getMessage());
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+
+    /**
+     * Creating a HBaseClient with the tableName null, should throw a exception
+     */
+    public void testCreatingTheHBaseClientWithTableNameNull() {
+        try {
+            new HBaseClient(getDataContext().getConnection()).dropTable(null);
+            fail("Should get an exception that tableName is null");
+        } catch (IllegalArgumentException e) {
+            assertEquals("Can't drop a table without having the tableName", e.getMessage());
+        }
+    }
+
+    /**
+     * Goodflow. Droping a table succesfully.
+     * @throws IOException
+     */
+    public void testDropTableSuccesfully() throws IOException {
+        if (isConfigured()) {
+            try {
+                final HBaseTable existingTable = createAndInsertTable(TABLE_NAME, HBaseDataContext.FIELD_ID, CF_FOO,
+                        CF_BAR);
+                getUpdateCallback().dropTable(existingTable).execute();
+                try (final Admin admin = getDataContext().getAdmin()) {
+                    assertFalse(admin.tableExists(TableName.valueOf(TABLE_NAME)));
+                }
+            } catch (Exception e) {
+                fail("Should not get an exception that the table doesn't exist in the datastore");
+            }
+        } else {
+            warnAboutANotExecutedTest(getClass().getName(), new Object() {
+            }.getClass().getEnclosingMethod().getName());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
index 3872611..e4e647c 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseDataContextTest.java
@@ -54,14 +54,13 @@ public class HBaseDataContextTest extends HBaseTestCase {
         assertEquals("[" + HBaseDataContext.FIELD_ID + ", " + CF_BAR + ", " + CF_FOO + "]", Arrays.toString(table
                 .getColumnNames()
                 .toArray()));
-        assertEquals(ColumnType.MAP, table.getColumn(1).getType());
+        assertEquals(HBaseColumn.DEFAULT_COLUMN_TYPE_FOR_COLUMN_FAMILIES, table.getColumn(1).getType());
 
         // insert two records
         insertRecordsNatively();
 
         // query using regular configuration
-        final DataSet dataSet1 = getDataContext().query().from(TABLE_NAME).selectAll().execute();
-        try {
+        try (final DataSet dataSet1 = getDataContext().query().from(TABLE_NAME).selectAll().execute()) {
             assertTrue(dataSet1.next());
             assertEquals("Row[values=[" + RK_1 + ", {" + Q_HEY + "=" + V_YO + "," + Q_HI + "=" + V_THERE + "}, {"
                     + Q_HELLO + "=" + V_WORLD + "}]]", dataSet1.getRow().toString());
@@ -69,8 +68,6 @@ public class HBaseDataContextTest extends HBaseTestCase {
             assertEquals("Row[values=[" + RK_2 + ", {" + Q_BAH + "=" + new String(V_123_BYTE_ARRAY) + "," + Q_HI + "="
                     + V_YOU + "}, {}]]", dataSet1.getRow().toString());
             assertFalse(dataSet1.next());
-        } finally {
-            dataSet1.close();
         }
 
         // query using custom table definitions
@@ -79,17 +76,16 @@ public class HBaseDataContextTest extends HBaseTestCase {
         final String columnName3 = CF_BAR + ":" + Q_HEY;
         final String[] columnNames = new String[] { columnName1, columnName2, columnName3 };
         final ColumnType[] columnTypes = new ColumnType[] { ColumnType.MAP, ColumnType.VARCHAR, ColumnType.VARCHAR };
-        final SimpleTableDef[] tableDefinitions = new SimpleTableDef[] { new SimpleTableDef(TABLE_NAME,
-                columnNames, columnTypes) };
+        final SimpleTableDef[] tableDefinitions = new SimpleTableDef[] { new SimpleTableDef(TABLE_NAME, columnNames,
+                columnTypes) };
         setDataContext(new HBaseDataContext(new HBaseConfiguration("SCH", getZookeeperHostname(), getZookeeperPort(),
                 tableDefinitions, ColumnType.VARCHAR)));
 
-        final DataSet dataSet2 = getDataContext()
+        try (final DataSet dataSet2 = getDataContext()
                 .query()
                 .from(TABLE_NAME)
                 .select(columnName1, columnName2, columnName3)
-                .execute();
-        try {
+                .execute()) {
             assertTrue(dataSet2.next());
             assertEquals("Row[values=[{" + Q_HELLO + "=" + V_WORLD + "}, " + V_THERE + ", " + V_YO + "]]", dataSet2
                     .getRow()
@@ -97,52 +93,39 @@ public class HBaseDataContextTest extends HBaseTestCase {
             assertTrue(dataSet2.next());
             assertEquals("Row[values=[{}, " + V_YOU + ", null]]", dataSet2.getRow().toString());
             assertFalse(dataSet2.next());
-        } finally {
-            dataSet2.close();
         }
 
         // query count
-        final DataSet dataSet3 = getDataContext().query().from(TABLE_NAME).selectCount().execute();
-        try {
+        try (final DataSet dataSet3 = getDataContext().query().from(TABLE_NAME).selectCount().execute()) {
             assertTrue(dataSet3.next());
             assertEquals("Row[values=[" + NUMBER_OF_ROWS + "]]", dataSet3.getRow().toString());
             assertFalse(dataSet3.next());
-        } finally {
-            dataSet3.close();
         }
 
         // query only id
-        final DataSet dataSet4 = getDataContext()
+        try (final DataSet dataSet4 = getDataContext()
                 .query()
                 .from(TABLE_NAME)
                 .select(HBaseDataContext.FIELD_ID)
-                .execute();
-
-        try {
+                .execute()) {
             assertTrue(dataSet4.next());
             assertEquals("Row[values=[" + RK_1 + "]]", dataSet4.getRow().toString());
             assertTrue(dataSet4.next());
             assertEquals("Row[values=[" + RK_2 + "]]", dataSet4.getRow().toString());
             assertFalse(dataSet4.next());
-        } finally {
-            dataSet4.close();
         }
 
         // primary key lookup query - using GET
-        final DataSet dataSet5 = getDataContext()
+        try (final DataSet dataSet5 = getDataContext()
                 .query()
                 .from(TABLE_NAME)
                 .select(HBaseDataContext.FIELD_ID)
                 .where(HBaseDataContext.FIELD_ID)
                 .eq(RK_1)
-                .execute();
-
-        try {
+                .execute()) {
             assertTrue(dataSet5.next());
             assertEquals("Row[values=[" + RK_1 + "]]", dataSet5.getRow().toString());
             assertFalse(dataSet5.next());
-        } finally {
-            dataSet5.close();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
index b9524a8..91180cb 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseTestCase.java
@@ -20,9 +20,11 @@ package org.apache.metamodel.hbase;
 
 import java.io.File;
 import java.io.FileReader;
+import java.io.IOException;
 import java.util.Properties;
 
 import org.apache.metamodel.schema.ColumnType;
+import org.junit.AfterClass;
 
 import junit.framework.TestCase;
 
@@ -59,33 +61,44 @@ public abstract class HBaseTestCase extends TestCase {
     private String zookeeperHostname;
     private int zookeeperPort;
     private boolean _configured;
-    private HBaseDataContext _dataContext;
+    private static HBaseDataContext _dataContext;
+
+    private boolean setUpIsDone = false;
 
     @Override
     protected void setUp() throws Exception {
         super.setUp();
 
-        Properties properties = new Properties();
-        File file = new File(getPropertyFilePath());
-        if (file.exists()) {
-            properties.load(new FileReader(file));
-            zookeeperHostname = properties.getProperty("hbase.zookeeper.hostname");
-            String zookeeperPortPropertyValue = properties.getProperty("hbase.zookeeper.port");
-            if (zookeeperPortPropertyValue != null && !zookeeperPortPropertyValue.isEmpty()) {
-                zookeeperPort = Integer.parseInt(zookeeperPortPropertyValue);
+        if (!setUpIsDone) {
+            Properties properties = new Properties();
+            File file = new File(getPropertyFilePath());
+            if (file.exists()) {
+                properties.load(new FileReader(file));
+                zookeeperHostname = properties.getProperty("hbase.zookeeper.hostname");
+                String zookeeperPortPropertyValue = properties.getProperty("hbase.zookeeper.port");
+                if (zookeeperPortPropertyValue != null && !zookeeperPortPropertyValue.isEmpty()) {
+                    zookeeperPort = Integer.parseInt(zookeeperPortPropertyValue);
+                }
+
+                _configured = (zookeeperHostname != null && !zookeeperHostname.isEmpty());
+            } else {
+                _configured = false;
             }
-
-            _configured = (zookeeperHostname != null && !zookeeperHostname.isEmpty());
-        } else {
-            _configured = false;
-        }
-        if (isConfigured()) {
-            final HBaseConfiguration configuration = new HBaseConfiguration(zookeeperHostname, zookeeperPort,
-                    ColumnType.VARCHAR);
-            setDataContext(new HBaseDataContext(configuration));
+            if (isConfigured()) {
+                final HBaseConfiguration configuration = new HBaseConfiguration(zookeeperHostname, zookeeperPort,
+                        ColumnType.VARCHAR);
+                setDataContext(new HBaseDataContext(configuration));
+            }
+            setUpIsDone = true;
         }
     }
 
+    @AfterClass
+    public static void oneTimeTeardown() throws IOException {
+        _dataContext.getConnection().close();
+        ;
+    }
+
     private String getPropertyFilePath() {
         String userHome = System.getProperty("user.home");
         return userHome + "/metamodel-integrationtest-configuration.properties";
@@ -113,12 +126,6 @@ public abstract class HBaseTestCase extends TestCase {
     }
 
     public void setDataContext(HBaseDataContext dataContext) {
-        this._dataContext = dataContext;
-    }
-
-    @Override
-    protected void tearDown() throws Exception {
-        super.tearDown();
-        _dataContext.getConnection().close();
+        HBaseTestCase._dataContext = dataContext;
     }
 }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/9ab30040/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
index 359be01..b010c71 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseUpdateCallbackTest.java
@@ -20,56 +20,62 @@ package org.apache.metamodel.hbase;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
-import java.util.Set;
+import java.util.List;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.metamodel.MetaModelException;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.metamodel.schema.ColumnType;
-import org.apache.metamodel.schema.ImmutableSchema;
 import org.apache.metamodel.schema.MutableSchema;
 import org.apache.metamodel.schema.Table;
 import org.apache.metamodel.util.SimpleTableDef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-public class HBaseUpdateCallbackTest extends HBaseTestCase {
+public abstract class HBaseUpdateCallbackTest extends HBaseTestCase {
+
+    private static final Logger logger = LoggerFactory.getLogger(HBaseClient.class);
 
     private HBaseUpdateCallback updateCallback;
     private MutableSchema schema;
 
+    private boolean setUpIsDone = false;
+
     @Override
     protected void setUp() throws Exception {
         super.setUp();
         if (isConfigured()) {
-            updateCallback = new HBaseUpdateCallback(getDataContext());
-            schema = (MutableSchema) getDataContext().getDefaultSchema();
-
-            if (schema.getTableByName(TABLE_NAME) != null) {
+            if (setUpIsDone) {
                 dropTableIfItExists();
+            } else {
+                updateCallback = new HBaseUpdateCallback(getDataContext());
+                schema = (MutableSchema) getDataContext().getDefaultSchema();
+                dropTableIfItExists();
+                setUpIsDone = true;
             }
         }
     }
 
-    public void testDropTable() throws IOException {
-        dropTableIfItExists();
-
-        try {
-            HBaseTable table = createHBaseTable();
-            updateCallback.dropTable(table).execute();
-            fail("Should get an exception that the table doesn't exist in the datastore");
-        } catch (MetaModelException e) {
-            assertEquals("Trying to delete a table that doesn't exist in the datastore.", e.getMessage());
+    @Override
+    public void tearDown() throws Exception {
+        if (isConfigured()) {
+            dropTableIfItExists();
         }
+        super.tearDown();
     }
 
-    private void dropTableIfItExists() {
-        Table table = schema.getTableByName(TABLE_NAME);
+    protected void dropTableIfItExists() {
+        final Table table = schema.getTableByName(TABLE_NAME);
         if (table != null) {
             updateCallback.dropTable(table).execute();
             // Check schema
             assertNull(schema.getTableByName(TABLE_NAME));
             // Check in the datastore
-            try (Admin admin = getDataContext().getAdmin()) {
+            try (final Admin admin = getDataContext().getAdmin()) {
                 assertFalse(admin.tableExists(TableName.valueOf(TABLE_NAME)));
             } catch (IOException e) {
                 fail("Should not an exception checking if the table exists");
@@ -77,137 +83,121 @@ public class HBaseUpdateCallbackTest extends HBaseTestCase {
         }
     }
 
-    public void testCreateTable() {
-        // Drop the table if it exists
-        dropTableIfItExists();
-
-        // Test 1: Create a table with an immutableSchema, should throw a IllegalArgumentException
-        ImmutableSchema immutableSchema = new ImmutableSchema(schema);
-        try {
-            updateCallback.createTable(immutableSchema, TABLE_NAME).execute();
-            fail("Should get an exception that the schema isn't mutable");
-        } catch (IllegalArgumentException e) {
-            assertEquals("Not a mutable schema: " + immutableSchema, e.getMessage());
+    protected void checkSuccesfullyInsertedTable() throws IOException {
+        // Check the schema
+        assertNotNull(schema.getTableByName(TABLE_NAME));
+        // Check in the datastore
+        try (final Admin admin = getDataContext().getAdmin()) {
+            assertTrue(admin.tableExists(TableName.valueOf(TABLE_NAME)));
+        } catch (IOException e) {
+            fail("Should not an exception checking if the table exists");
         }
+    }
 
-        // Test 2: Create a table without columnFamilies, should throw a MetaModelException
-        try {
-            updateCallback.createTable(schema, TABLE_NAME).execute();
-            fail("Should get an exception that the columnFamilies haven't been set");
-        } catch (MetaModelException e) {
-            assertEquals("Creating a table without columnFamilies", e.getMessage());
-        }
+    protected HBaseTable createAndInsertTable(final String tableName, final String idColumn, final String columnFamily1,
+            final String columnFamily2) throws IOException {
+        final LinkedHashSet<String> columnFamilies = new LinkedHashSet<>();
+        columnFamilies.add(idColumn);
+        columnFamilies.add(columnFamily1);
+        columnFamilies.add(columnFamily2);
+        updateCallback.createTable(schema, tableName, columnFamilies).execute();
+        checkSuccesfullyInsertedTable();
+        return (HBaseTable) getDataContext().getDefaultSchema().getTableByName(tableName);
+    }
 
-        // Test 3: Create a table with columnFamilies null, should throw a MetaModelException
-        try {
-            updateCallback.createTable(schema, TABLE_NAME, null).execute();
-            fail("Should get an exception that the columnFamilies haven't been set");
-        } catch (MetaModelException e) {
-            assertEquals("Creating a table without columnFamilies", e.getMessage());
+    protected HBaseTable createHBaseTable(final String tableName, final String idColumn, final String columnFamily1,
+            final String columnFamily2, final String columnFamily3) {
+        String[] columnNames;
+        ColumnType[] columnTypes;
+        if (columnFamily3 == null) {
+            columnNames = new String[] { idColumn, columnFamily1, columnFamily2 };
+            columnTypes = new ColumnType[] { ColumnType.STRING, ColumnType.STRING, ColumnType.STRING };
+        } else {
+            columnNames = new String[] { idColumn, columnFamily1, columnFamily2, columnFamily3 };
+            columnTypes = new ColumnType[] { ColumnType.STRING, ColumnType.STRING, ColumnType.STRING,
+                    ColumnType.STRING };
         }
+        final SimpleTableDef tableDef = new SimpleTableDef(tableName, columnNames, columnTypes);
+        return new HBaseTable(getDataContext(), tableDef, schema, ColumnType.STRING);
+    }
 
-        // Test 4: Create a table with columnFamilies empty, should throw a MetaModelException
-        try {
-            final LinkedHashSet<String> columnFamilies = new LinkedHashSet<String>();
-            updateCallback.createTable(schema, TABLE_NAME, columnFamilies).execute();
-            fail("Should get an exception that the columnFamilies haven't been set");
-        } catch (MetaModelException e) {
-            assertEquals("Creating a table without columnFamilies", e.getMessage());
-        }
+    protected static LinkedHashMap<HBaseColumn, Object> createRow(final HBaseTable table, final String idColumn,
+            final String columnFamily1, final String columnFamily2) {
+        final LinkedHashMap<HBaseColumn, Object> map = new LinkedHashMap<>();
 
-        HBaseTable table = createHBaseTable();
-
-        // Test 5: Create a table without the ID-Column, should throw a MetaModelException
-        ArrayList<HBaseColumn> hBaseColumnsAsArrayList = createListWithHBaseColumnsExcludingIDColumn(table);
-        HBaseColumn[] hBaseColumnsAsArray = convertToHBaseColumnArray(hBaseColumnsAsArrayList);
-        Set<String> columnFamilies = HBaseColumn.getColumnFamilies(hBaseColumnsAsArray);
-        try {
-            HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) updateCallback.createTable(
-                    schema, TABLE_NAME);
-
-            hBaseCreateTableBuilder.setColumnFamilies(columnFamilies);
-            hBaseCreateTableBuilder.execute();
-            fail("Should get an exception that the ID-colum is missing");
-        } catch (MetaModelException e) {
-            assertEquals("ColumnFamily: " + HBaseDataContext.FIELD_ID + " not found", e.getMessage());
+        // Columns
+        final ArrayList<HBaseColumn> columns = new ArrayList<>();
+        if (idColumn != null) {
+            columns.add(new HBaseColumn(idColumn, table));
         }
-
-        // Test 6: Create a table including the ID-Column (columnFamilies not in constructor), should work
-        hBaseColumnsAsArrayList = createListWithHBaseColumnsIncludingIDColumn(table);
-        hBaseColumnsAsArray = convertToHBaseColumnArray(hBaseColumnsAsArrayList);
-        columnFamilies = HBaseColumn.getColumnFamilies(hBaseColumnsAsArray);
-        try {
-            HBaseCreateTableBuilder hBaseCreateTableBuilder = (HBaseCreateTableBuilder) updateCallback.createTable(
-                    schema, TABLE_NAME);
-
-            hBaseCreateTableBuilder.setColumnFamilies(HBaseColumn.getColumnFamilies(hBaseColumnsAsArray));
-            hBaseCreateTableBuilder.execute();
-            checkSuccesfullyInsertedTable();
-        } catch (Exception e) {
-            fail("Should not get an exception");
+        columns.add(new HBaseColumn(columnFamily1, Q_HELLO, table));
+        columns.add(new HBaseColumn(columnFamily1, Q_HI, table));
+        columns.add(new HBaseColumn(columnFamily2, Q_HEY, table));
+        columns.add(new HBaseColumn(columnFamily2, Q_BAH, table));
+
+        // Values
+        final ArrayList<Object> values = new ArrayList<>();
+        if (idColumn != null) {
+            values.add(RK_1);
         }
-        dropTableIfItExists();
-
-        // Test 7: Create a table including the ID-Column (columnFamilies in constructor), should work
-        try {
-            updateCallback.createTable(schema, TABLE_NAME, columnFamilies).execute();
-            checkSuccesfullyInsertedTable();
-        } catch (Exception e) {
-            fail("Should not get an exception");
+        values.add(V_WORLD);
+        values.add(V_THERE);
+        values.add(V_YO);
+        values.add(V_123_BYTE_ARRAY);
+
+        // Fill the map
+        for (int i = 0; i < columns.size(); i++) {
+            map.put(columns.get(i), values.get(i));
         }
-        dropTableIfItExists();
+
+        return map;
     }
 
-    private void checkSuccesfullyInsertedTable() throws IOException {
-        // Check the schema
-        assertNotNull(schema.getTableByName(TABLE_NAME));
-        // Check in the datastore
-        try (Admin admin = getDataContext().getAdmin()) {
-            assertTrue(admin.tableExists(TableName.valueOf(TABLE_NAME)));
-        } catch (IOException e) {
-            fail("Should not an exception checking if the table exists");
-        }
+    protected static List<HBaseColumn> getHBaseColumnsFromMap(final LinkedHashMap<HBaseColumn, Object> map) {
+        final List<HBaseColumn> columns = new ArrayList<>();
+        columns.addAll(map.keySet());
+        return columns;
     }
 
-    // public void testInsertRows() throws IOException {
-    // // Drop the table if it exists
-    // dropTableIfItExists();
-    //
-    // insertTable();
-    // }
-
-    private void insertTable() throws IOException {
-        HBaseTable table = createHBaseTable();
-        ArrayList<HBaseColumn> hBaseColumnsAsArrayList = createListWithHBaseColumnsIncludingIDColumn(table);
-        HBaseColumn[] hBaseColumnsAsArray = convertToHBaseColumnArray(hBaseColumnsAsArrayList);
-        Set<String> columnFamilies = HBaseColumn.getColumnFamilies(hBaseColumnsAsArray);
-        updateCallback.createTable(schema, TABLE_NAME, columnFamilies).execute();
-        checkSuccesfullyInsertedTable();
+    protected void setValuesInInsertionBuilder(final LinkedHashMap<HBaseColumn, Object> row,
+            final HBaseRowInsertionBuilder rowInsertionBuilder) {
+        int i = 0;
+        for (Object value : row.values()) {
+            rowInsertionBuilder.value(i, value);
+            i++;
+        }
     }
 
-    private HBaseTable createHBaseTable() {
-        String[] columnNames = new String[] { CF_FOO, CF_BAR };
-        ColumnType[] columnTypes = new ColumnType[] { ColumnType.STRING, ColumnType.STRING };
-        SimpleTableDef tableDef = new SimpleTableDef(TABLE_NAME, columnNames, columnTypes);
-        return new HBaseTable(getDataContext(), tableDef, schema, ColumnType.STRING);
+    protected void checkRows(final boolean rowsExist) throws IOException {
+        try (org.apache.hadoop.hbase.client.Table table = getDataContext().getConnection().getTable(TableName.valueOf(
+                TABLE_NAME))) {
+            final Get get = new Get(Bytes.toBytes(RK_1));
+            final Result result = table.get(get);
+            if (rowsExist) {
+                assertFalse(result.isEmpty());
+                assertEquals(V_WORLD, new String(result.getValue(Bytes.toBytes(CF_FOO), Bytes.toBytes(Q_HELLO))));
+                assertEquals(V_THERE, new String(result.getValue(Bytes.toBytes(CF_FOO), Bytes.toBytes(Q_HI))));
+                assertEquals(V_YO, new String(result.getValue(Bytes.toBytes(CF_BAR), Bytes.toBytes(Q_HEY))));
+                assertEquals(V_123_BYTE_ARRAY.toString(), new String(result.getValue(Bytes.toBytes(CF_BAR), Bytes
+                        .toBytes(Q_BAH))));
+            } else {
+                assertTrue(result.isEmpty());
+            }
+        }
     }
 
-    private static ArrayList<HBaseColumn> createListWithHBaseColumnsExcludingIDColumn(final HBaseTable table) {
-        ArrayList<HBaseColumn> hbaseColumns = new ArrayList<HBaseColumn>();
-        hbaseColumns.add(new HBaseColumn(CF_FOO, Q_HELLO, table));
-        hbaseColumns.add(new HBaseColumn(CF_FOO, Q_HI, table));
-        hbaseColumns.add(new HBaseColumn(CF_BAR, Q_HEY, table));
-        hbaseColumns.add(new HBaseColumn(CF_BAR, Q_BAH, table));
-        return hbaseColumns;
+    protected void warnAboutANotExecutedTest(String className, String methodName) {
+        String logWarning = "Test(method) \"" + className + "#" + methodName
+                + "\" is not executed, because the HBasetest is not configured.";
+        // System.out.println(logWarning);
+        logger.warn(logWarning);
     }
 
-    private static ArrayList<HBaseColumn> createListWithHBaseColumnsIncludingIDColumn(final HBaseTable table) {
-        ArrayList<HBaseColumn> hbaseColumns = createListWithHBaseColumnsExcludingIDColumn(table);
-        hbaseColumns.add(new HBaseColumn(HBaseDataContext.FIELD_ID, table));
-        return hbaseColumns;
+    protected HBaseUpdateCallback getUpdateCallback() {
+        return updateCallback;
     }
 
-    private static HBaseColumn[] convertToHBaseColumnArray(final ArrayList<HBaseColumn> hBaseColumnsAsArrayList) {
-        return hBaseColumnsAsArrayList.toArray(new HBaseColumn[hBaseColumnsAsArrayList.size()]);
+    protected MutableSchema getSchema() {
+        return schema;
     }
 }


[23/31] metamodel git commit: Reverted back to private as it was before.

Posted by ka...@apache.org.
Reverted back to private as it was before.


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/575ffbf2
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/575ffbf2
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/575ffbf2

Branch: refs/heads/master
Commit: 575ffbf262766942fbdaf8103547f1ce5df84add
Parents: e37546d
Author: Arjan Seijkens <a....@quadient.com>
Authored: Fri Jun 8 15:17:49 2018 +0200
Committer: Arjan Seijkens <a....@quadient.com>
Committed: Fri Jun 8 15:17:49 2018 +0200

----------------------------------------------------------------------
 .../src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/575ffbf2/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
index 9c989db..ec91d8f 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java
@@ -93,7 +93,7 @@ public class HBaseDataContext extends QueryPostprocessDataContext implements Upd
         }
     }
 
-    protected static Configuration createConfig(HBaseConfiguration configuration) {
+    private static Configuration createConfig(HBaseConfiguration configuration) {
         Configuration config = org.apache.hadoop.hbase.HBaseConfiguration.create();
         config.set("hbase.zookeeper.quorum", configuration.getZookeeperHostname());
         config.set("hbase.zookeeper.property.clientPort", Integer.toString(configuration.getZookeeperPort()));


[06/31] metamodel git commit: Made getColumnsInternal method synchronized to prevent ConcurrentModificationExceptions when columns are retrieved from the Table in an parallel manner.

Posted by ka...@apache.org.
Made getColumnsInternal method synchronized to prevent ConcurrentModificationExceptions when columns are retrieved from the Table in an parallel manner.


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/38d8bc3e
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/38d8bc3e
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/38d8bc3e

Branch: refs/heads/master
Commit: 38d8bc3e18c6f966c2ef0fe661532d177e9359ee
Parents: 0ffeeb2
Author: arjansh <Ar...@humaninference.com>
Authored: Wed May 16 13:39:26 2018 +0200
Committer: arjansh <Ar...@humaninference.com>
Committed: Wed May 16 13:39:26 2018 +0200

----------------------------------------------------------------------
 hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/38d8bc3e/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
----------------------------------------------------------------------
diff --git a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
index 03c3263..def7c9c 100644
--- a/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
+++ b/hbase/src/main/java/org/apache/metamodel/hbase/HBaseTable.java
@@ -92,7 +92,7 @@ final class HBaseTable extends MutableTable {
     }
 
     @Override
-    protected List<Column> getColumnsInternal() {
+    protected synchronized List<Column> getColumnsInternal() {
         final List<Column> columnsInternal = super.getColumnsInternal();
         if (columnsInternal.isEmpty() && _dataContext != null) {
             try {


[24/31] metamodel git commit: Overriding setUp method with @Before annotation to make sure setUp method on parent class is invoked before running tests.

Posted by ka...@apache.org.
Overriding setUp method with @Before annotation to make sure setUp method on parent class is invoked before running tests.


Project: http://git-wip-us.apache.org/repos/asf/metamodel/repo
Commit: http://git-wip-us.apache.org/repos/asf/metamodel/commit/146277c3
Tree: http://git-wip-us.apache.org/repos/asf/metamodel/tree/146277c3
Diff: http://git-wip-us.apache.org/repos/asf/metamodel/diff/146277c3

Branch: refs/heads/master
Commit: 146277c3c475356496808152e0f59a963b0de222
Parents: 575ffbf
Author: Arjan Seijkens <a....@quadient.com>
Authored: Mon Jun 11 13:38:48 2018 +0200
Committer: Arjan Seijkens <a....@quadient.com>
Committed: Mon Jun 11 13:38:48 2018 +0200

----------------------------------------------------------------------
 .../test/java/org/apache/metamodel/hbase/HBaseClientTest.java | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/146277c3/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java
----------------------------------------------------------------------
diff --git a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java
index 35aa310..31d6a0d 100644
--- a/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java
+++ b/hbase/src/test/java/org/apache/metamodel/hbase/HBaseClientTest.java
@@ -21,6 +21,7 @@ package org.apache.metamodel.hbase;
 import java.util.LinkedHashSet;
 import java.util.Set;
 
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -29,6 +30,12 @@ public class HBaseClientTest extends HBaseTestCase {
     @Rule
     public ExpectedException exception = ExpectedException.none();
 
+    @Override
+    @Before
+    public void setUp() throws Exception {
+        super.setUp();
+    }
+
     /**
      * Creating a HBaseClient with the tableName null, should throw a exception
      */