You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@metamodel.apache.org by GerardDellemann <gi...@git.apache.org> on 2018/06/06 08:26:02 UTC

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

GitHub user GerardDellemann opened a pull request:

    https://github.com/apache/metamodel/pull/182

    Add Create, Update, Delete functionality for HBase

    Create: It's possible to create tables and insert rows.
    Update: Insert and update are the same in HBase.
    Delete: It's possible to drop a table and delete rows based on the rowKey.

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

    $ git pull https://github.com/GerardDellemann/metamodel feature/hbase-improvements

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

    https://github.com/apache/metamodel/pull/182.patch

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

    This closes #182
    
----
commit 6ad602bb9af70ed7214b6d7e4f00cd1d9a037f71
Author: HUMANINFERENCE\g.dellemann <g....@...>
Date:   2018-04-20T14:36:27Z

    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.

commit f15d02721910678e29c1452c1bf6f4b495609efc
Author: HUMANINFERENCE\g.dellemann <g....@...>
Date:   2018-04-23T14:53:03Z

    Small improvements and fixed the existing Unit Test

commit 2008c7eb641c647ffe2588a1119b5d52f8fc9809
Author: arjansh <ar...@...>
Date:   2018-04-30T09:49:52Z

    Reverted changes to logic in core module and refactored HBaseColumn class so it now extends the AbstractColumn class (and implement the Column class).

commit f2d2ff2a503ceebbd0bd3493d4df60cae08126ae
Author: Gerard Dellemann <ge...@...>
Date:   2018-04-30T14:08:06Z

    Merge pull request #1 from arjansh/feature/hbase-improvements-refactoring
    
    Reverting some core changes and refactoring HBaseColumn class

commit 0ffeeb20af4d58061d042bb6ebccafb2ab7d6502
Author: Gerard Dellemann <g....@...>
Date:   2018-05-01T12:34:36Z

    HBase improvements - Added row-deletion functionality, also made HBaseDataContext implement UpdateableDataContext

commit 38d8bc3e18c6f966c2ef0fe661532d177e9359ee
Author: arjansh <ar...@...>
Date:   2018-05-16T11:39:26Z

    Made getColumnsInternal method synchronized to prevent ConcurrentModificationExceptions when columns are retrieved from the Table in an parallel manner.

commit 3ada7eb48e0599b507024b1b6c10be3b7cceb63f
Author: Gerard Dellemann <g....@...>
Date:   2018-05-24T14:12:20Z

    HBase improvements: refactoring, new unittests

commit ea7356a4228f700b1b8f7115762cfc42ca6c92fc
Author: arjansh <ar...@...>
Date:   2018-05-25T10:08:05Z

    Refactored HBaseRowInsertionBuilder, so it uses the _columns of the AbstractRowBuilder instead of its own, to make sure the model stays consistent.

commit 86084869ab7eefb92d983b472fb4664f3a0f742d
Author: arjansh <ar...@...>
Date:   2018-05-25T11:12:36Z

    Removed table exists check. Let's assume the createTable message throws an exception when it fails to create the table.

commit 9ab30040e3311f5c9947615470ad1d366de92c0f
Author: Gerard Dellemann <g....@...>
Date:   2018-05-29T13:28:27Z

    HBase Improvements - did some refactoring, added comments, added unittest

commit 54a900b4c9a608687239982d602c8eabc67f3afa
Author: Gerard Dellemann <g....@...>
Date:   2018-05-30T08:54:22Z

    HBase unittests improvements (fixed issues when the testfile wasn't setup)

commit 5a28d73cdddfc8b11ca2478e390c8c94bcf34310
Author: Arjan Seijkens <a....@...>
Date:   2018-05-31T08:38:13Z

    - 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.

commit 703fb40e80a91521aca62fae918737aaa5e752bb
Author: Gerard Dellemann <ge...@...>
Date:   2018-05-31T11:11:08Z

    Merge pull request #3 from GerardDellemann/feature/hbase-improvements-refactoring
    
    Refactoring code and implementing feedback

commit cd755a1cf5ea3762463c577dd220162dee14dfd9
Author: Gerard Dellemann <g....@...>
Date:   2018-05-31T12:10:13Z

    HBase improvements (small refactorings, add comments)

commit f4d2c97ad3a7fa41934251e6fd727639ed1bd300
Author: Gerard Dellemann <g....@...>
Date:   2018-05-31T14:30:20Z

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

commit 717a3a443624f2da1a883a26d17a25886c77d67e
Author: Arjan Seijkens <a....@...>
Date:   2018-06-01T09:25:29Z

    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.

commit f48e9c833f574663a29078ba67b87f6932685287
Author: Gerard Dellemann <ge...@...>
Date:   2018-06-03T08:44:17Z

    Merge pull request #4 from GerardDellemann/feature/hbase-improvements-assume-in-tests
    
    Refactored to remove boiler plate isConfigured checks

commit 4a928d175897f6d71738a7113663723d160ffa0b
Author: Gerard Dellemann <g....@...>
Date:   2018-06-03T23:20:09Z

    HBase improvements - Refactoring (mostly small improvements, the getHBaseClient method has been moved to HBaseDataContent)

commit 5313a492f7c733be367c2738af14bc566a6090e1
Author: Arjan Seijkens <a....@...>
Date:   2018-06-06T08:12:02Z

    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

----


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193617545
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java ---
    @@ -0,0 +1,112 @@
    +/**
    + * 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.List;
    +import java.util.Set;
    +
    +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.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 {
    +
    +    public HBaseUpdateCallback(final HBaseDataContext dataContext) {
    +        super(dataContext);
    +    }
    +
    +    @Override
    +    public TableCreationBuilder createTable(final Schema schema, final String name) {
    +        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;
    +    }
    +
    +    @Override
    +    public TableDropBuilder dropTable(final Table table) {
    +        return new HBaseTableDropBuilder(table, this);
    +    }
    +
    +    /**
    +     * @throws UnsupportedOperationException use {@link HBaseUpdateCallback#insertInto(String, String)}
    +     */
    +    @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) {
    --- End diff --
    
    I don't think it makes much sense to expose this method which is not part of the interface, but to make the `insertInto(Table)` method throw `UnsupportedOperationException`. If the user cannot use the interface, then why even implement it? The user will have to do something quite awkward like:
    
    ```
    dataContext.executeUpdate(cb -> {
      HBaseUpdateCallback hbc = (HBaseUpdateCallback) cb;
      // something goes here
    });
    ```
    Normally we even keep the UpdateCallback implementations non-public because we don't want the users to know about it, just the interface.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193792898
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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.Set;
    +
    +import org.apache.metamodel.MetaModelException;
    +import org.apache.metamodel.create.AbstractTableCreationBuilder;
    +import org.apache.metamodel.schema.MutableSchema;
    +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;
    +
    +    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}.
    +     * @param updateCallback
    +     * @param schema
    +     * @param name
    +     * @param 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);
    +        }
    +        this._columnFamilies = columnFamilies;
    +    }
    +
    +    @Override
    +    public Table execute() {
    +        if (_columnFamilies == null || _columnFamilies.isEmpty()) {
    --- End diff --
    
    I see that we have `DEFAULT_COLUMN_TYPE_FOR_COLUMN_FAMILIES`. So maybe we could provide this way of creating column/column families:
    
    ```
    Callback cb = ...
    cb.createTable(schema, "My_table")
      .withColumn("family1").ofType(DEFAULT_COLUMN_TYPE_FOR_COLUMN_FAMILIES)
      .withColumn("family2.foo").ofType(STRING)
      .withColumn("family2.bar").ofType(INTEGER)
    ```
    
    What that would amount to is two column families created: `family1` and `family2`. The schema for `family1` would not be defined, so it would just be represented as a single MM Column with type LIST, but the schema for `family2` will have been defined to contain `foo` and `bar`. Just like how we can "define" it via a `SimpleTableDef`.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r195067558
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java ---
    @@ -106,9 +105,9 @@ public String toString() {
                 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()));
    --- End diff --
    
    The problem however is that the `Arrays.toString` variant returns the key/value combination of `hey` and `yo` as `[104, 101, 121]` and `[121, 111]`, which you can hardly call a String representation. where the `new String` will typically return `hey` and `yo` in that case. In case some unexpected character encoding is used it may display some shady characters, but it should still work.


---

[GitHub] metamodel issue #182: Add Create, Update, Delete functionality for HBase

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

    https://github.com/apache/metamodel/pull/182
  
    Done, build is working now


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193617724
  
    --- Diff: core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java ---
    @@ -44,6 +44,12 @@ public AbstractRowInsertionBuilder(U updateCallback, Table table) {
             _table = table;
         }
     
    +    public AbstractRowInsertionBuilder(final U updateCallback, final Table table, final List<Column> columns) {
    --- End diff --
    
    I don't think this constructor will be needed if we resolve my first discussion point. And because of that, I prefer to not add it, because not having it keeps the check in place that we don't suddenly start requiring a different method signature wherever this constructor is called.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193618035
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java ---
    @@ -0,0 +1,124 @@
    +/**
    + * 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.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);
    --- End diff --
    
    Actually you may want to just use `ColumnType.BINARY` instead of creating a new ColumnTypeImpl here.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193663410
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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.Set;
    +
    +import org.apache.metamodel.MetaModelException;
    +import org.apache.metamodel.create.AbstractTableCreationBuilder;
    +import org.apache.metamodel.schema.MutableSchema;
    +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;
    +
    +    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}.
    +     * @param updateCallback
    +     * @param schema
    +     * @param name
    +     * @param 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);
    +        }
    +        this._columnFamilies = columnFamilies;
    +    }
    +
    +    @Override
    +    public Table execute() {
    +        if (_columnFamilies == null || _columnFamilies.isEmpty()) {
    --- End diff --
    
    Qualifiers are not part of table's schema. They are not used at creating a table.


---

[GitHub] metamodel issue #182: Add Create, Update, Delete functionality for HBase

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

    https://github.com/apache/metamodel/pull/182
  
    Oh I think that's a glitch. I've seen that test fail on rare occasions before. @GerardDellemann you can trigger a new build by making an empty commit to your branch, like this:
    ```
    git commit --allow-empty -m "Empty commit to trigger rebuild"
    ```


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193618132
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java ---
    @@ -0,0 +1,124 @@
    +/**
    + * 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.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_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);
    +    }
    +
    +    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) {
    --- End diff --
    
    This is a lot of constructors :-) I would only add the ones you use yourself, and then make the class non-public and final.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r194912353
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java ---
    @@ -242,17 +240,18 @@ protected DataSet materializeMainSchemaTable(Table table, List<Column> columns,
         }
     
         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);
    --- End diff --
    
    Is using `Scan.setMaxResultSize(...)` still not the recommended approach to setting max rows? If so, we should keep this approach. I think the PageFilter is considerably slower.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193787182
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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.Set;
    +
    +import org.apache.metamodel.MetaModelException;
    +import org.apache.metamodel.create.AbstractTableCreationBuilder;
    +import org.apache.metamodel.schema.MutableSchema;
    +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;
    +
    +    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}.
    +     * @param updateCallback
    +     * @param schema
    +     * @param name
    +     * @param 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);
    +        }
    +        this._columnFamilies = columnFamilies;
    +    }
    +
    +    @Override
    +    public Table execute() {
    +        if (_columnFamilies == null || _columnFamilies.isEmpty()) {
    --- End diff --
    
    An option might be to drop the setColumnFamilies method and instead use the withColumn method to set the column families for this CreateTableBuilder, but then we have to make clear that the column added by the withColumn method represents a column family for the HBase perspective.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r195638498
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java ---
    @@ -0,0 +1,200 @@
    +/**
    + * 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.ArrayList;
    +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.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.
    + */
    +public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
    +    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.
    +     * @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) {
    +        super(updateCallback, table);
    +
    +        checkTable(updateCallback, table);
    +    }
    +
    +    /**
    +     * 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(
    +                tableGettingInserts.getName());
    +        if (tableInSchema == null) {
    +            throw new MetaModelException("Trying to insert data into table: " + tableGettingInserts.getName()
    +                    + ", which doesn't exist yet");
    +        }
    +        checkColumnFamilies(tableInSchema, tableGettingInserts.getColumnFamilies());
    +    }
    +
    +    /**
    +     * Check if a list of columnNames all exist in this table
    +     * @param table Checked tabled
    +     * @param columnFamilyNamesOfCheckedTable
    +     * @throws MetaModelException If a column doesn't exist
    +     */
    +    public void checkColumnFamilies(final HBaseTable table, final Set<String> columnFamilyNamesOfCheckedTable) {
    +        Set<String> columnFamilyNamesOfExistingTable = table.getColumnFamilies();
    +
    +        for (String columnNameOfCheckedTable : columnFamilyNamesOfCheckedTable) {
    +            boolean matchingColumnFound = false;
    +            Iterator<String> columnFamilies = columnFamilyNamesOfExistingTable.iterator();
    +            while (!matchingColumnFound && columnFamilies.hasNext()) {
    +                if (columnNameOfCheckedTable.equals(columnFamilies.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 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
    +    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.");
    +        }
    +
    +        final HBaseColumn hbaseColumn = getHbaseColumn(column);
    +
    +        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();
    +        }
    +
    +        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) {
    --- End diff --
    
    I'm going to look a bit into the necessity of the using the abstract class, maybe a full implementation is indeed better in this case.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r195718876
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java ---
    @@ -0,0 +1,200 @@
    +/**
    + * 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.ArrayList;
    +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.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.
    + */
    +public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
    +    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.
    +     * @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) {
    +        super(updateCallback, table);
    +
    +        checkTable(updateCallback, table);
    +    }
    +
    +    /**
    +     * 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(
    +                tableGettingInserts.getName());
    +        if (tableInSchema == null) {
    +            throw new MetaModelException("Trying to insert data into table: " + tableGettingInserts.getName()
    +                    + ", which doesn't exist yet");
    +        }
    +        checkColumnFamilies(tableInSchema, tableGettingInserts.getColumnFamilies());
    +    }
    +
    +    /**
    +     * Check if a list of columnNames all exist in this table
    +     * @param table Checked tabled
    +     * @param columnFamilyNamesOfCheckedTable
    +     * @throws MetaModelException If a column doesn't exist
    +     */
    +    public void checkColumnFamilies(final HBaseTable table, final Set<String> columnFamilyNamesOfCheckedTable) {
    +        Set<String> columnFamilyNamesOfExistingTable = table.getColumnFamilies();
    +
    +        for (String columnNameOfCheckedTable : columnFamilyNamesOfCheckedTable) {
    +            boolean matchingColumnFound = false;
    +            Iterator<String> columnFamilies = columnFamilyNamesOfExistingTable.iterator();
    +            while (!matchingColumnFound && columnFamilies.hasNext()) {
    +                if (columnNameOfCheckedTable.equals(columnFamilies.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 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
    +    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.");
    +        }
    +
    +        final HBaseColumn hbaseColumn = getHbaseColumn(column);
    +
    +        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();
    +        }
    +
    +        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) {
    --- End diff --
    
    I refactored the class so it no longer extend the `AbstractRowInsertionBuilder`, but implements its logic completely by itself.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193786120
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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.Set;
    +
    +import org.apache.metamodel.MetaModelException;
    +import org.apache.metamodel.create.AbstractTableCreationBuilder;
    +import org.apache.metamodel.schema.MutableSchema;
    +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;
    +
    +    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}.
    +     * @param updateCallback
    +     * @param schema
    +     * @param name
    +     * @param 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);
    +        }
    +        this._columnFamilies = columnFamilies;
    +    }
    +
    +    @Override
    +    public Table execute() {
    +        if (_columnFamilies == null || _columnFamilies.isEmpty()) {
    --- End diff --
    
    When you create an HBase table, you specify the Column Families for the tables. After the table has been created the Column Families are fixed and can't be changed. Within each column family you have the freedom to write what you want, so qualifiers aren't part of the creation of the table, the are created on the fly when writing data to an HBase table. Therefore we only mention column families and not qualifiers or a combination of both when creating the Table.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r195045275
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java ---
    @@ -242,17 +240,18 @@ protected DataSet materializeMainSchemaTable(Table table, List<Column> columns,
         }
     
         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);
    --- End diff --
    
    The max result size is not related to the number of returned rows. It sets the number of bytes which is returned by a single scan before it scans for more. So if you set it to 1, a scan will only return 1 byte and if a row contains of more then 1 byte it will just return the complete row and after that do a new scan for the next row.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r194912619
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java ---
    @@ -106,9 +105,9 @@ public String toString() {
                 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()));
    --- End diff --
    
    Isn't this problematic, since the byte-array may not be a string, and it may not even be decodeable as a string, so this could throw.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r196366302
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java ---
    @@ -0,0 +1,207 @@
    +/**
    + * 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
    + */
    +final class HBaseClient {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(HBaseClient.class);
    +
    +    private final Connection _connection;
    +
    +    public HBaseClient(final Connection connection) {
    +        this._connection = connection;
    +    }
    +
    +    /**
    +     * Insert a single row of values to a HBase table.
    +     * @param tableName
    +     * @param columns
    +     * @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 caught
    +     */
    +    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(getValueAsByteArray(values[indexOfIdColumn]));
    +
    +            // Add the other values to the put
    +            for (int i = 0; i < columns.length; i++) {
    +                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
    +                    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]);
    +                    // 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());
    --- End diff --
    
    It is possible that an HBaseColumn doesn't have a qualifier, in which case this will throw a NullPointerException, so either do `Bytes.toString(qualifier)`, or an inline if to check if the qualifier is null and in that case return "" for it.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r195718665
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java ---
    @@ -106,9 +105,9 @@ public String toString() {
                 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()));
    --- End diff --
    
    The Apache Hadoop Hbase library has a utility method for this `org.apache.hadoop.hbase.util.Bytes#toString(byte[])`, so I changed it to use that instead.


---

[GitHub] metamodel issue #182: Add Create, Update, Delete functionality for HBase

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

    https://github.com/apache/metamodel/pull/182
  
    However, the Travis build seems to have failed :-)


---

[GitHub] metamodel issue #182: Add Create, Update, Delete functionality for HBase

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

    https://github.com/apache/metamodel/pull/182
  
    I think the behavior of HBase is mostly similar to Cassandra and I do think that we can improve on it in the future. We haven't addressed the reading from HBase in this PR and that still has lots of room for improvement from my perspective, I think adding that annotation at least indicates we expect to evolve this in the future, so I agree.


---

[GitHub] metamodel issue #182: Add Create, Update, Delete functionality for HBase

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

    https://github.com/apache/metamodel/pull/182
  
    I like what this PR does, but it seems a bit one-off to me. It's nice to see us supporting on-the-fly column definitions during insert, but the same principle could also apply for other connectors we have, such as the ones for CouchDB, MongoDB, ElasticSearch and more. So since the approach is a bit bespoke for HBase, and we don't know if there will be some consolidation later if we end up implementing something a bit more generic for these types of datastores, I think for now we should add the `@InterfaceStability.Evolving` annotation to the HBaseDataContext. What do you think about that?


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r195406132
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java ---
    @@ -106,9 +105,9 @@ public String toString() {
                 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()));
    --- End diff --
    
    I believe that there are byte combinations that are simply invalid. So I suspect it could throw exceptions in certain cases. I'm not an expert on this however. Anyone else who has certainty?


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r195064147
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java ---
    @@ -0,0 +1,200 @@
    +/**
    + * 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.ArrayList;
    +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.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.
    + */
    +public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
    +    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.
    +     * @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) {
    +        super(updateCallback, table);
    +
    +        checkTable(updateCallback, table);
    +    }
    +
    +    /**
    +     * 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(
    +                tableGettingInserts.getName());
    +        if (tableInSchema == null) {
    +            throw new MetaModelException("Trying to insert data into table: " + tableGettingInserts.getName()
    +                    + ", which doesn't exist yet");
    +        }
    +        checkColumnFamilies(tableInSchema, tableGettingInserts.getColumnFamilies());
    +    }
    +
    +    /**
    +     * Check if a list of columnNames all exist in this table
    +     * @param table Checked tabled
    +     * @param columnFamilyNamesOfCheckedTable
    +     * @throws MetaModelException If a column doesn't exist
    +     */
    +    public void checkColumnFamilies(final HBaseTable table, final Set<String> columnFamilyNamesOfCheckedTable) {
    +        Set<String> columnFamilyNamesOfExistingTable = table.getColumnFamilies();
    +
    +        for (String columnNameOfCheckedTable : columnFamilyNamesOfCheckedTable) {
    +            boolean matchingColumnFound = false;
    +            Iterator<String> columnFamilies = columnFamilyNamesOfExistingTable.iterator();
    +            while (!matchingColumnFound && columnFamilies.hasNext()) {
    +                if (columnNameOfCheckedTable.equals(columnFamilies.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 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
    +    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.");
    +        }
    +
    +        final HBaseColumn hbaseColumn = getHbaseColumn(column);
    +
    +        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();
    +        }
    +
    +        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) {
    --- End diff --
    
    The special reason is that we don't use the `_values` and `_columns` arrays used by the super classes, because when creating the insertion builder we can only pass the Table in the constructor and the available columns on an HBase table aren't fixed, because qualifiers are dynamic can be added on the fly, only the column families are fixed.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r195404505
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseDataContext.java ---
    @@ -242,17 +240,18 @@ protected DataSet materializeMainSchemaTable(Table table, List<Column> columns,
         }
     
         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);
    --- End diff --
    
    Oh, good to know :-) Then the change certainly makes sense.


---

[GitHub] metamodel issue #182: Add Create, Update, Delete functionality for HBase

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

    https://github.com/apache/metamodel/pull/182
  
    Merged.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193617943
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java ---
    @@ -0,0 +1,124 @@
    +/**
    + * 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.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);
    --- End diff --
    
    Should probably be `SuperColumnType.BINARY_TYPE`


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193784059
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java ---
    @@ -0,0 +1,124 @@
    +/**
    + * 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.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);
    --- End diff --
    
    I changed it to ColumnType.BINARY.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193785301
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseColumn.java ---
    @@ -0,0 +1,124 @@
    +/**
    + * 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.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_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);
    +    }
    +
    +    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) {
    --- End diff --
    
    I removed the unused constructors. For now I left the class public however, because it represents the manner in which an HBase column works. I think it is possible to use a Column, but then you need some logic to deduce the actual HBase column from the normal Column, using a colon to split the Column's name on and get an HBaseColumn based on that.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r194911581
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java ---
    @@ -0,0 +1,197 @@
    +/**
    + * 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
    + */
    +final class HBaseClient {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(HBaseClient.class);
    +
    +    private final Connection _connection;
    +
    +    public HBaseClient(final Connection connection) {
    +        this._connection = connection;
    +    }
    +
    +    /**
    +     * Insert a single row of values to a HBase table.
    +     * @param tableName
    +     * @param columns
    +     * @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 caught
    +     */
    +    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(getValueAsByteArray(values[indexOfIdColumn]));
    +
    +            // Add the other values to the put
    +            for (int i = 0; i < columns.length; i++) {
    +                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
    +                    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
    +            table.put(put);
    +        } catch (IOException e) {
    +            throw new MetaModelException(e);
    +        }
    +    }
    +
    +    /**
    +     * Delete 1 row based on the key
    +     * @param tableName
    +     * @param rowKey
    +     * @throws IllegalArgumentException when any parameter is null
    +     * @throws MetaModelException when a {@link IOException} is caught
    +     */
    +    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 = getValueAsByteArray(rowKey);
    +        if (rowKeyAsByteArray.length > 0) {
    +            try (final Table table = _connection.getTable(TableName.valueOf(tableName))) {
    +                if (rowExists(table, rowKeyAsByteArray)) {
    +                    table.delete(new Delete(rowKeyAsByteArray));
    +                } else {
    +                    logger.warn("Rowkey with value {} doesn't exist in the table", rowKey.toString());
    +                }
    +            } catch (IOException e) {
    +                throw new MetaModelException(e);
    +            }
    +        } else {
    +            logger.info("Have not deleted a row, which has an empty (\"\") rowKey.");
    --- End diff --
    
    Should we maybe throw here instead? Or at the least log at error level. But I think throwing makes most sense.


---

[GitHub] metamodel issue #182: Add Create, Update, Delete functionality for HBase

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

    https://github.com/apache/metamodel/pull/182
  
    Assuming that the build passes now and all that, this all LGTM!


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193783877
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseUpdateCallback.java ---
    @@ -0,0 +1,112 @@
    +/**
    + * 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.List;
    +import java.util.Set;
    +
    +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.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 {
    +
    +    public HBaseUpdateCallback(final HBaseDataContext dataContext) {
    +        super(dataContext);
    +    }
    +
    +    @Override
    +    public TableCreationBuilder createTable(final Schema schema, final String name) {
    +        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;
    +    }
    +
    +    @Override
    +    public TableDropBuilder dropTable(final Table table) {
    +        return new HBaseTableDropBuilder(table, this);
    +    }
    +
    +    /**
    +     * @throws UnsupportedOperationException use {@link HBaseUpdateCallback#insertInto(String, String)}
    +     */
    +    @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) {
    --- End diff --
    
    This part has been refactored. You can now use the insertInto(Table) method as exposed by the api and add the columns and values to the returned RowInsertionBuilder.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r196363308
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseClient.java ---
    @@ -0,0 +1,207 @@
    +/**
    + * 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
    + */
    +final class HBaseClient {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(HBaseClient.class);
    +
    +    private final Connection _connection;
    +
    +    public HBaseClient(final Connection connection) {
    +        this._connection = connection;
    +    }
    +
    +    /**
    +     * Insert a single row of values to a HBase table.
    +     * @param tableName
    +     * @param columns
    +     * @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 caught
    +     */
    +    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(getValueAsByteArray(values[indexOfIdColumn]));
    +
    +            // Add the other values to the put
    +            for (int i = 0; i < columns.length; i++) {
    +                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
    +                    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]);
    +                    // 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]
    --- End diff --
    
    Can you log this at `info` level instead? 


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193618625
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseCreateTableBuilder.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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.Set;
    +
    +import org.apache.metamodel.MetaModelException;
    +import org.apache.metamodel.create.AbstractTableCreationBuilder;
    +import org.apache.metamodel.schema.MutableSchema;
    +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;
    +
    +    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}.
    +     * @param updateCallback
    +     * @param schema
    +     * @param name
    +     * @param 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);
    +        }
    +        this._columnFamilies = columnFamilies;
    +    }
    +
    +    @Override
    +    public Table execute() {
    +        if (_columnFamilies == null || _columnFamilies.isEmpty()) {
    --- End diff --
    
    Instead of requiring column families to be called out in a way that's not part of MetaModel's public API, could we do something smart with e.g. column names that represent the column families? For example, we could use a dot-separated column name approach, where "family1.foo" and "family1.bar" represented two column in the same column family. I think we would need this in order to retain compatibility with the public API. I'm not a big fan of introducing HBase-specific methods in `UpdateCallback` since we've then kinda gone against the idea of using MetaModel in the first place (a uniform API for database access).


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r195406963
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java ---
    @@ -0,0 +1,200 @@
    +/**
    + * 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.ArrayList;
    +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.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.
    + */
    +public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
    +    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.
    +     * @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) {
    +        super(updateCallback, table);
    +
    +        checkTable(updateCallback, table);
    +    }
    +
    +    /**
    +     * 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(
    +                tableGettingInserts.getName());
    +        if (tableInSchema == null) {
    +            throw new MetaModelException("Trying to insert data into table: " + tableGettingInserts.getName()
    +                    + ", which doesn't exist yet");
    +        }
    +        checkColumnFamilies(tableInSchema, tableGettingInserts.getColumnFamilies());
    +    }
    +
    +    /**
    +     * Check if a list of columnNames all exist in this table
    +     * @param table Checked tabled
    +     * @param columnFamilyNamesOfCheckedTable
    +     * @throws MetaModelException If a column doesn't exist
    +     */
    +    public void checkColumnFamilies(final HBaseTable table, final Set<String> columnFamilyNamesOfCheckedTable) {
    +        Set<String> columnFamilyNamesOfExistingTable = table.getColumnFamilies();
    +
    +        for (String columnNameOfCheckedTable : columnFamilyNamesOfCheckedTable) {
    +            boolean matchingColumnFound = false;
    +            Iterator<String> columnFamilies = columnFamilyNamesOfExistingTable.iterator();
    +            while (!matchingColumnFound && columnFamilies.hasNext()) {
    +                if (columnNameOfCheckedTable.equals(columnFamilies.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 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
    +    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.");
    +        }
    +
    +        final HBaseColumn hbaseColumn = getHbaseColumn(column);
    +
    +        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();
    +        }
    +
    +        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) {
    --- End diff --
    
    In that case, shouldn't we then avoid the abstract class, and instead do a full implementation of the interface?


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r194913069
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java ---
    @@ -0,0 +1,200 @@
    +/**
    + * 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.ArrayList;
    +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.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.
    + */
    +public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
    +    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.
    +     * @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) {
    +        super(updateCallback, table);
    +
    +        checkTable(updateCallback, table);
    +    }
    +
    +    /**
    +     * 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(
    +                tableGettingInserts.getName());
    +        if (tableInSchema == null) {
    +            throw new MetaModelException("Trying to insert data into table: " + tableGettingInserts.getName()
    +                    + ", which doesn't exist yet");
    +        }
    +        checkColumnFamilies(tableInSchema, tableGettingInserts.getColumnFamilies());
    +    }
    +
    +    /**
    +     * Check if a list of columnNames all exist in this table
    +     * @param table Checked tabled
    +     * @param columnFamilyNamesOfCheckedTable
    +     * @throws MetaModelException If a column doesn't exist
    +     */
    +    public void checkColumnFamilies(final HBaseTable table, final Set<String> columnFamilyNamesOfCheckedTable) {
    +        Set<String> columnFamilyNamesOfExistingTable = table.getColumnFamilies();
    +
    +        for (String columnNameOfCheckedTable : columnFamilyNamesOfCheckedTable) {
    +            boolean matchingColumnFound = false;
    +            Iterator<String> columnFamilies = columnFamilyNamesOfExistingTable.iterator();
    +            while (!matchingColumnFound && columnFamilies.hasNext()) {
    +                if (columnNameOfCheckedTable.equals(columnFamilies.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 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
    +    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.");
    +        }
    +
    +        final HBaseColumn hbaseColumn = getHbaseColumn(column);
    +
    +        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();
    +        }
    +
    +        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) {
    --- End diff --
    
    When you extend the abstract class that you use here, you shouldn't need to override these methods. Is there some special reason for doing that? I worry that any new method addition will be forgotten here and would rather that we just used the abstract class'es methods.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r195409088
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java ---
    @@ -0,0 +1,200 @@
    +/**
    + * 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.ArrayList;
    +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.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.
    + */
    +public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
    +    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.
    +     * @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) {
    +        super(updateCallback, table);
    +
    +        checkTable(updateCallback, table);
    +    }
    +
    +    /**
    +     * 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(
    +                tableGettingInserts.getName());
    +        if (tableInSchema == null) {
    +            throw new MetaModelException("Trying to insert data into table: " + tableGettingInserts.getName()
    +                    + ", which doesn't exist yet");
    +        }
    +        checkColumnFamilies(tableInSchema, tableGettingInserts.getColumnFamilies());
    +    }
    +
    +    /**
    +     * Check if a list of columnNames all exist in this table
    +     * @param table Checked tabled
    +     * @param columnFamilyNamesOfCheckedTable
    +     * @throws MetaModelException If a column doesn't exist
    +     */
    +    public void checkColumnFamilies(final HBaseTable table, final Set<String> columnFamilyNamesOfCheckedTable) {
    +        Set<String> columnFamilyNamesOfExistingTable = table.getColumnFamilies();
    +
    +        for (String columnNameOfCheckedTable : columnFamilyNamesOfCheckedTable) {
    +            boolean matchingColumnFound = false;
    +            Iterator<String> columnFamilies = columnFamilyNamesOfExistingTable.iterator();
    +            while (!matchingColumnFound && columnFamilies.hasNext()) {
    +                if (columnNameOfCheckedTable.equals(columnFamilies.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 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
    +    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.");
    +        }
    +
    +        final HBaseColumn hbaseColumn = getHbaseColumn(column);
    +
    +        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();
    +        }
    +
    +        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) {
    --- End diff --
    
    Wait ... So how would a user be able to pass in a column that's not already defined? I see that `HBaseTable` returns `HBaseColumn`s which represent that paths down into the column families as defined by the `SimpleTableDef`s. So it's kind of dynamic based on that `SimpleTableDef`. But is there some way that the columns will change after creating the simple table def? Or rather - in the middle of a update script? I don't see how that would be possible, but maybe it is, and then I think that deserves some clarification (since it's unusual for MetaModel).


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r193783947
  
    --- Diff: core/src/main/java/org/apache/metamodel/insert/AbstractRowInsertionBuilder.java ---
    @@ -44,6 +44,12 @@ public AbstractRowInsertionBuilder(U updateCallback, Table table) {
             _table = table;
         }
     
    +    public AbstractRowInsertionBuilder(final U updateCallback, final Table table, final List<Column> columns) {
    --- End diff --
    
    This constructor has been removed.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r195638392
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseRowInsertionBuilder.java ---
    @@ -0,0 +1,200 @@
    +/**
    + * 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.ArrayList;
    +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.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.
    + */
    +public class HBaseRowInsertionBuilder extends AbstractRowInsertionBuilder<HBaseUpdateCallback> {
    +    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.
    +     * @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) {
    +        super(updateCallback, table);
    +
    +        checkTable(updateCallback, table);
    +    }
    +
    +    /**
    +     * 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(
    +                tableGettingInserts.getName());
    +        if (tableInSchema == null) {
    +            throw new MetaModelException("Trying to insert data into table: " + tableGettingInserts.getName()
    +                    + ", which doesn't exist yet");
    +        }
    +        checkColumnFamilies(tableInSchema, tableGettingInserts.getColumnFamilies());
    +    }
    +
    +    /**
    +     * Check if a list of columnNames all exist in this table
    +     * @param table Checked tabled
    +     * @param columnFamilyNamesOfCheckedTable
    +     * @throws MetaModelException If a column doesn't exist
    +     */
    +    public void checkColumnFamilies(final HBaseTable table, final Set<String> columnFamilyNamesOfCheckedTable) {
    +        Set<String> columnFamilyNamesOfExistingTable = table.getColumnFamilies();
    +
    +        for (String columnNameOfCheckedTable : columnFamilyNamesOfCheckedTable) {
    +            boolean matchingColumnFound = false;
    +            Iterator<String> columnFamilies = columnFamilyNamesOfExistingTable.iterator();
    +            while (!matchingColumnFound && columnFamilies.hasNext()) {
    +                if (columnNameOfCheckedTable.equals(columnFamilies.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 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
    +    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.");
    +        }
    +
    +        final HBaseColumn hbaseColumn = getHbaseColumn(column);
    +
    +        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();
    +        }
    +
    +        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) {
    --- End diff --
    
    The columns are not defined, only the column families are, qualifiers are not. So you can insert values into columns which don't exist yet. Each row in an HBase table can have different columns from other rows in that table. Effectively that makes it harder to read from an HBase table in a manner that fits MetaModel then to write to an HBase table, because MetaModel typically writes the to same columns over and over again and that's no problem for HBase.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182#discussion_r195639023
  
    --- Diff: hbase/src/main/java/org/apache/metamodel/hbase/HBaseFamilyMap.java ---
    @@ -106,9 +105,9 @@ public String toString() {
                 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()));
    --- End diff --
    
    The behavior of this String constructor when the given bytes are not valid in the default charset is unspecified. There is another String constructor which also takes a Charset as an arguments, that method always replaces malformed-input and unmappable-character sequences with the charset's default replacement string. So maybe we should use that instead.


---

[GitHub] metamodel pull request #182: Add Create, Update, Delete functionality for HB...

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

    https://github.com/apache/metamodel/pull/182


---