You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hawq.apache.org by leskin-in <gi...@git.apache.org> on 2018/02/26 16:55:06 UTC

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

GitHub user leskin-in opened a pull request:

    https://github.com/apache/incubator-hawq/pull/1344

    PXF Ignite plugin

    Add a new PXF Ignite plugin to access (read & write) [Ignite database](https://ignite.apache.org/) via REST API.
    
    Documentation is available at `pxf/pxf-ignite/README.md`.
    
    Key features:
    
    * Supports partitioning (similar to one in PXF JDBC plugin);
    
    * Uses a dedicated Ignite client with REST API enabled. The location of the client is a customizable PXF user property;
    
    * Every SELECT or INSERT query splits tuples of data into groups, each of which is send from (to) Ignite in one response (query). The size of a group is a customizable PXF user property;
    
    * This REST API can be modified (with relatively small changes) and used to connect to any other database (or service) which supports SQL queries via REST, and responds using JSON objects.
    
    [GSON](https://github.com/google/gson) is used to parse JSON objects. No other extra dependencies (except for those already present in HAWQ PXF) are required.


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

    $ git pull https://github.com/arenadata/incubator-hawq pxf_ignite_plugin

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

    https://github.com/apache/incubator-hawq/pull/1344.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 #1344
    
----
commit 80c547c5a00f34526889dc405c839e3765b29c6e
Author: Ivan Leskin <le...@...>
Date:   2018-02-08T13:11:19Z

    Add PXF-Ignite plug-in.
    
    The PXF-Ignite plug-in allows to read data from Ignite database.
    The syntax is similar to one of PXF-JDBC plug-in.
    
    This implementation uses Ignite REST API. It can be easily modified to access any other databases or services via REST API.
    
    This commit does not change `pxf-service` links and classpath files. They should be added manually in order for the PXF-Ignite to work.
    
    See pxf-ignite/README.md for instructions on how to use this plug-in.

commit 8cb62b751674be0878247eb95f01ecc236a4f7f2
Author: Ivan Leskin <le...@...>
Date:   2018-02-08T14:19:37Z

    PXF Ignite plug-in: Implement write access
    
    This commit adds the support of write operations to the PXF Ignite plug-in.

commit 920615d1e7435003beb5fe7a758eb62da3219c8f
Author: Ivan Leskin <le...@...>
Date:   2018-02-26T15:00:00Z

    PXF Ignite plug-in: tests and fixes

----


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171036360
  
    --- Diff: pxf/pxf-ignite/README.md ---
    @@ -0,0 +1,67 @@
    +# Accessing Ignite database using PXF
    +
    +The PXF Ignite plug-in enables to access the [Ignite database](https://ignite.apache.org/) (both read and write operations are supported) via REST API.
    +
    +
    +# Prerequisites
    +
    +Check the following before using the plug-in:
    +
    +* The Ignite plug-in is installed on all PXF nodes;
    +
    +* The Ignite client is installed and running at the `IGNITE_HOST` (`localhost` by default; see below), and it accepts http queries from the PXF (note that *enabling Ignite REST API does not require changes in Ignite configuration*; see the instruction on how to do that at https://apacheignite.readme.io/docs/rest-api#section-getting-started).
    +
    +
    +# Syntax
    +
    +```
    +CREATE [READABLE] EXTERNAL TABLE <table_name> (
    +    <column_name> <data_type>[, <column_name> <data_type>, ...] | LIKE <other_table>
    +)
    +LOCATION ('pxf://<ignite_table_name>?PROFILE=Ignite[&<extra-parameter>&<extra-parameter>&...]')
    +FORMAT 'CUSTOM' (formatter='pxfwritable_import');
    +```
    +where each `<extra-parameter>` is one of the following:
    +* `IGNITE_HOST=<ignite_host_address_with_port>`. The location of Ignite client node. If not given, `127.0.0.1:8080` is used by default;
    +* `IGNITE_CACHE=<ignite_cache_name>`. The name of Ignite cache to use. If not given, this parameter is not included in queries from PXF to Ignite, thus Ignite default values will be used (at the moment, this is `Default` cache). This option is **case-sensitive**;
    +* `BUFFER_SIZE=<unsigned_int>`. The number of tuples send to (from) Ignite per a response. The same number of tuples is stored in in-plug-in cache. The values `0` and `1` are equal (cache is not used, each tuple is passed in it's own query to Ignite). If not given, `128` is used by default;
    +* `PARTITION_BY=<column>:<column_type>`. See below;
    +* `RANGE=<start_value>:<end_value>`. See below;
    +* `INTERVAL=<value>[:<unit>]`. See below.
    +
    +
    +# Partitioning
    +## Introduction
    +
    +PXF Ignite plugin supports simultaneous access to Ignite database from multiple PXF segments. *Partitioning* should be used in order to perform such operation.
    +
    +If the partitioning is not used, all the data will be retrieved by a single PXF segment.
    +
    +Partitioning in PXF Ignite plug-in works just like in PXF JDBC plug-in.
    +
    +This feature is optional. However, a bug in the `pxf-service` which makes partitioning necessary for any query was fixed only on 17th Jan 2018 in [this commit](https://github.com/apache/incubator-hawq/commit/0d620e431026834dd70c9e0d63edf8bb28b38227), so the older versions of PXF may return an exception if a query does not contain a meaningful `PARTITION_BY` parameter.
    --- End diff --
    
    I don't think we need to include this statement, as this new plugin cannot be built with the older PXF version in the open source.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171332966
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteResolver.java ---
    @@ -0,0 +1,173 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +
    +import java.util.ArrayList;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.text.SimpleDateFormat;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.codec.binary.Base64;
    +import org.apache.commons.codec.binary.Hex;
    +
    +import com.google.gson.JsonArray;
    +
    +
    --- End diff --
    
    Is it correct to put two new lines after all the imports?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171319392
  
    --- Diff: pxf/pxf-ignite/src/test/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessorTest.java ---
    @@ -0,0 +1,130 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import com.google.gson.JsonArray;
    +import com.google.gson.JsonObject;
    +
    +import org.junit.Test;
    +import org.junit.Before;
    +import org.junit.runner.RunWith;
    +import static org.junit.Assert.*;
    +
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Mockito;
    +import static org.mockito.Matchers.anyString;
    +
    +import org.powermock.api.mockito.PowerMockito;
    +import org.powermock.core.classloader.annotations.PrepareForTest;
    +import org.powermock.modules.junit4.PowerMockRunner;
    +import static org.powermock.api.support.membermodification.MemberMatcher.method;
    +
    +
    +@RunWith(PowerMockRunner.class)
    +@PrepareForTest({IgniteAccessor.class})
    +public class IgniteAccessorTest {
    +    @Before
    +    public void prepareAccessorTest() throws Exception {
    +        inputData = Mockito.mock(InputData.class);
    +
    +        Mockito.when(inputData.getDataSource()).thenReturn("TableTest");
    +
    +        columns.add(new ColumnDescriptor("id", DataType.INTEGER.getOID(), 0, "int4", null));
    +        columns.add(new ColumnDescriptor("name", DataType.TEXT.getOID(), 1, "text", null));
    +        columns.add(new ColumnDescriptor("birthday", DataType.DATE.getOID(), 2, "date", null));
    +        columns.add(new ColumnDescriptor("key", DataType.BYTEA.getOID(), 3, "bytea", null));
    +        Mockito.when(inputData.getTupleDescription()).thenReturn(columns);
    +        Mockito.when(inputData.getColumn(0)).thenReturn(columns.get(0));
    +        Mockito.when(inputData.getColumn(1)).thenReturn(columns.get(1));
    +        Mockito.when(inputData.getColumn(2)).thenReturn(columns.get(2));
    +        Mockito.when(inputData.getColumn(3)).thenReturn(columns.get(3));
    +    }
    +
    +    @Test
    +    public void testReadAccess() throws Exception {
    +        IgniteAccessor acc = PowerMockito.spy(new IgniteAccessor(inputData));
    +
    +        JsonObject correctAnswer = new JsonObject();
    +        JsonArray tempArray = new JsonArray();
    +        JsonArray tempArray2 = new JsonArray();
    +        tempArray2.add(1);
    +        tempArray2.add("abcd");
    +        tempArray2.add("'2001-01-01'");
    +        tempArray2.add("YWJjZA==");
    +        tempArray.add(tempArray2);
    +        correctAnswer.add("items", tempArray);
    +        correctAnswer.addProperty("last", false);
    +        correctAnswer.addProperty("queryId", 1);
    +
    +        ArgumentCaptor<String> captor = ArgumentCaptor.forClass(String.class);
    +        PowerMockito.doReturn(correctAnswer).when(acc, "sendRestRequest", anyString());
    +
    +        acc.openForRead();
    +        acc.readNextObject();
    +        acc.closeForRead();
    +
    +        PowerMockito.verifyPrivate(acc, Mockito.times(3)).invoke(method(IgniteAccessor.class, "sendRestRequest", String.class)).withArguments(captor.capture());
    +
    +        List<String> allParams = captor.getAllValues();
    +
    +        assertEquals(allParams.get(0), "http://127.0.0.1:8080/ignite?cmd=qryfldexe&pageSize=0&qry=SELECT+id%2Cname%2Cbirthday%2Ckey+FROM+TableTest");
    --- End diff --
    
    When `java.net.URLEncoder.encode()` is called [here in IgniteAccessor](https://github.com/arenadata/incubator-hawq/blob/06a287693f6bdeef72c43d606aeebf44a18ced98/pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java#L342)


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171331419
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    +        return true;
    +    }
    +
    +    /**
    +     * readNextObject() implementation
    +     */
    +    @Override
    +    public OneRow readNextObject() throws Exception {
    +        if (urlReadFetch == null) {
    +            LOG.error("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +            throw new ProtocolException("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +        }
    +
    +        if (bufferRead.isEmpty()) {
    +            // Refill buffer
    +            if (isLastReadFinished) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +
    +            JsonElement response = sendRestRequest(urlReadFetch);
    +            isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +
    +            // Parse 'items'
    +            Iterator<JsonElement> itemsIterator = response.getAsJsonObject().get("items").getAsJsonArray().iterator();
    +            while (itemsIterator.hasNext()) {
    +                if (!bufferRead.add(itemsIterator.next().getAsJsonArray())) {
    +                    LOG.error("readNextObject(): Buffer refill failed (not enough memory in 'bufferRead')");
    +                    throw new OutOfMemoryError("readNextObject(): not enough memory in 'bufferRead'");
    +                }
    +            }
    +
    +            // Check again in case "response" contains no elements
    +            if (bufferRead.isEmpty()) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): Buffer refill failed");
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +        }
    +
    +        return new OneRow(bufferRead.pollFirst());
    +    }
    +
    +    /**
    +     * closeForRead() implementation
    +     */
    +    @Override
    +    public void closeForRead() {
    +        if (urlReadClose != null) {
    +            try {
    +                sendRestRequest(urlReadClose);
    +            }
    +            catch (Exception e) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("closeForRead() Exception: " + e.getClass().getSimpleName());
    +                }
    +            }
    +        }
    +        isLastReadFinished = false;
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Ignite read request finished. URL: '" + urlReadClose + "'");
    +        }
    +    }
    +
    +    /**
    +     * openForWrite() implementation.
    +     * No queries are sent to Ignite by this procedure, so if there are some problems (for example, with connection), they will be revealed only during the execution of 'writeNextObject()'
    +     */
    +    @Override
    +    public boolean openForWrite() throws UserDataException {
    +        // This is a temporary solution. At the moment there is no other way (except for the usage of user-defined parameters) to get the correct name of Ignite table: GPDB inserts extra data into the address, as required by Hadoop.
    +        // Note that if no extra data is present, the 'definedSource' will be left unchanged
    +        String definedSource = inputData.getDataSource();
    +        Pattern pattern = Pattern.compile("/(.*)/[0-9]*-[0-9]*_[0-9]*");
    +        Matcher matcher = pattern.matcher(definedSource);
    +        if (matcher.find()) {
    +            inputData.setDataSource(matcher.group(1));
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("INSERT INTO ");
    +
    +        // Insert the table name
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert the column names
    +        sb.append("(");
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        String fieldDivisor = "";
    +        for (int i = 0; i < columns.size(); i++) {
    +            sb.append(fieldDivisor);
    +            fieldDivisor = ", ";
    +            sb.append(columns.get(i).columnName());
    +        }
    +        sb.append(")");
    +
    +        sb.append(" VALUES ");
    +
    +        queryWrite = sb.toString();
    +        return true;
    +    }
    +
    +    /**
    +     * writeNextObject() implementation
    +     */
    +    @Override
    +    public boolean writeNextObject(OneRow currentRow) throws Exception {
    +        boolean currentRowInBuffer = bufferWrite.add(currentRow);
    +
    +        if (!isWriteActive) {
    +            if (!currentRowInBuffer) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferWrite')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferWrite'");
    +            }
    +            LOG.info("Ignite write request. Query: '" + queryWrite + "'");
    +            sendInsertRestRequest(queryWrite);
    +            bufferWrite.removeFirst();
    +            isWriteActive = true;
    +            return true;
    +        }
    +
    +        if ((bufferWrite.size() >= bufferSize) || (!currentRowInBuffer)) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +
    +        if (!currentRowInBuffer) {
    +            if (!bufferWrite.add(currentRow)) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferSend')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferSend'");
    +            }
    +        }
    +        
    +        return true;
    +    }
    +
    +    /**
    +     * closeForWrite() implementation
    +     */
    +    @Override
    +    public void closeForWrite() throws Exception {
    +        if (!bufferWrite.isEmpty()) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +        if (isWriteActive) {
    +            // At this point, the request must have finished successfully; otherwise an exception would be thrown before
    +            LOG.info("Ignite write request finished successfully. Query: '" + queryWrite + "'");
    +        }
    +        isWriteActive = false;
    +    }
    +
    +    /**
    +     * Build HTTP GET query for Ignite REST API with command 'qryfldexe'
    +     * 
    +     * @param querySql SQL query with filter constants. The constants must replaced by "?" if 'filterConstants' is not given
    +     * @param filterConstants
    +     * 
    +     * @return Prepared HTTP query. The query will be properly encoded with {@link java.net.URLEncoder}
    +     * 
    +     * @throws UnsupportedEncodingException from {@link java.net.URLEncoder.encode()}
    +     */
    +    private String buildQueryFldexe(String querySql, ArrayList<String> filterConstants) throws UnsupportedEncodingException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("http://");
    +        sb.append(igniteHost);
    +        sb.append("/ignite");
    +        sb.append("?");
    +        sb.append("cmd=qryfldexe");
    +        sb.append("&");
    +        sb.append("pageSize=0");
    +        sb.append("&");
    +        if (cacheName != null) {
    +            sb.append("cacheName=");
    +            // Note that Ignite supports only "good" cache names (those that should be left unchanged by the URLEncoder.encode())
    +            sb.append(URLEncoder.encode(cacheName, "UTF-8"));
    +            sb.append("&");
    +        }
    +        int counter = 1;
    +        if (filterConstants != null) {
    --- End diff --
    
    This behaviour is correct.
    
    Apache Ignite [supports](https://apacheignite.readme.io/docs/rest-api#section-sql-fields-query-execute) passing the constraints' constants separately from the constraint expression itself. I am not sure why there is such an option; maybe it is there for safety (to protect Ignite from SQL injections, if the request is formed by, say, javascript on some webpage).
    
    This feature may be implemented in the future, so I left the code as it is. To implement the feature, relatively serious changes must be done both in `IgnitePartitionFragmenter` and `WhereSqlBuilder`; however, the effect of those changes is not evident.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171129868
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,384 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +/**
    + * Fragmenter class for Ignite data resources.
    + * @note At the moment this fragmenter works just like the JDBC's one
    + *
    + * Aside from implementing PXF {@link Fragmenter},
    + * this class also transforms input data path (given by user) by adding the
    + * information on splitting the request into a few requests for different regions.
    + * <br>
    + * The parameter Patterns
    + * <br>
    + * There are three  parameters, the format is as follows:
    + * <br>
    + * <pre>
    + * <code>PARTITION_BY=column_name:column_type&amp;RANGE=start_value[:end_value]&amp;INTERVAL=interval_num[:interval_unit]</code>
    + * </pre>
    + * The <code>PARTITION_BY</code> parameter can be split by colon(':'),the <code>column_type</code> current supported : <code>date,int,enum</code> .
    + * The Date format is 'yyyy-MM-dd'. <br>
    + * The <code>RANGE</code> parameter can be split by colon(':') ,used to identify the starting range of each fragment.
    + * The range is left-closed, ie:<code> '&gt;= start_value AND &lt; end_value' </code>.If the <code>column_type</code> is <code>int</code>,
    + * the <code>end_value</code> can be empty. If the <code>column_type</code>is <code>enum</code>,the parameter <code>RANGE</code> can be empty. <br>
    + * The <code>INTERVAL</code> parameter can be split by colon(':'), indicate the interval value of one fragment.
    + * When <code>column_type</code> is <code>date</code>,this parameter must be split by colon, and <code>interval_unit</code> can be <code>year,month,day</code>.
    + * When <code>column_type</code> is <code>int</code>, the <code>interval_unit</code> can be empty.
    + * When <code>column_type</code> is <code>enum</code>,the <code>INTERVAL</code> parameter can be empty.
    + * <br>
    + * <p>
    + * The syntax examples is:
    + * <br>
    + * <code>PARTITION_BY=createdate:date&amp;RANGE=2008-01-01:2010-01-01&amp;INTERVAL=1:month'</code>
    + * <br>
    + * <code>PARTITION_BY=year:int&amp;RANGE=2008:2010&amp;INTERVAL=1</code>
    + * <br>
    + * <code>PARTITION_BY=grade:enum&amp;RANGE=excellent:good:general:bad</code>
    + * </p>
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    Log LOG = LogFactory.getLog(IgnitePartitionFragmenter.class);
    +
    +    String[] partitionBy = null;
    +    String[] range = null;
    +    String[] interval = null;
    +    PartitionType partitionType = null;
    +    String partitionColumn = null;
    +    IntervalType intervalType = null;
    +    int intervalNum = 1;
    +
    +    //when partitionType is DATE,it is valid
    +    Calendar rangeStart = null;
    +    Calendar rangeEnd = null;
    +
    +
    +    enum PartitionType {
    +        DATE,
    +        INT,
    +        ENUM;
    +
    +        public static PartitionType getType(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    enum IntervalType {
    +        DAY,
    +        MONTH,
    +        YEAR;
    +
    +        public static IntervalType type(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    /**
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inConf) throws UserDataException {
    +        super(inConf);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inConf.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inConf.getUserProperty("PARTITION_BY").split(":");
    +            partitionColumn = partitionBy[0];
    +            partitionType = PartitionType.getType(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' invalid, the pattern is 'column_name:date|int|enum'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inConf.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inConf.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.type(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    +                }
    +            } 
    +            else if (partitionType != PartitionType.ENUM) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be specified along with 'PARTITION_BY'");
    +            }
    +            if (intervalNum < 1) {
    +                throw new UserDataException("The parameter 'INTERVAL' must > 1, but actual is '" + intervalNum + "'");
    --- End diff --
    
    typo `must be`, `actual value`


---

[GitHub] incubator-hawq issue #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344
  
    @kapustor To get this PR merged, we need this PR have two '+1' from the reviewer, and you need to create corresponding jira and include the jira number in the title of the PR. You can refer to other PRs. Thanks.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171499457
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,411 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +
    +/**
    + * PXF-Ignite fragmenter class
    + * 
    + * This fragmenter works just like the one in PXF JDBC plugin
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    /**
    +     * Class constructor
    +     * 
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inputData) throws UserDataException {
    +        super(inputData);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inputData.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inputData.getUserProperty("PARTITION_BY").split(":");
    +            partitionType = PartitionType.typeOf(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' is invalid. The pattern is 'column_name:DATE|INT|ENUM'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inputData.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' is invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inputData.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.typeOf(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    +                }
    +            } 
    +            else if (partitionType != PartitionType.ENUM) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be specified along with 'PARTITION_BY'");
    +            }
    +            if (intervalNum < 1) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be at least 1. The actual is '" + intervalNum + "'");
    +            }
    +        } 
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'INTERVAL' invalid. The pattern is 'interval_num[:interval_unit]'");
    +        }
    +
    +        //parse date values
    +        try {
    +            if (partitionType == PartitionType.DATE) {
    +                SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd");
    +                rangeStart = Calendar.getInstance();
    +                rangeStart.setTime(df.parse(range[0]));
    +                rangeEnd = Calendar.getInstance();
    +                rangeEnd.setTime(df.parse(range[1]));
    +            }
    +        } catch (ParseException e) {
    +            throw new UserDataException("The parameter 'RANGE' has invalid date format. Expected format is 'yyyy-MM-dd'");
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful; some partition used");
    +        }
    +    }
    +
    +    /**
    +     * Returns statistics for the Ignite table. This is not implemented in the current version
    +     * @throws UnsupportedOperationException
    +     */
    +    @Override
    +    public FragmentsStats getFragmentsStats() throws UnsupportedOperationException {
    +        throw new UnsupportedOperationException("ANALYZE for Ignite plugin is not supported");
    +    }
    +
    +    /**
    +     * Returns list of fragments for Ignite table queries
    +     *
    +     * @throws UnsupportedOperationException if a partition of unknown type was found
    +     * 
    +     * @return a list of fragments
    +     */
    +    @Override
    +    public List<Fragment> getFragments() throws UnsupportedOperationException {
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() called; dataSource is '" + inputData.getDataSource() + "'");
    +        }
    +
    +        byte[] fragmentMetadata = null;
    +        byte[] fragmentUserdata = null;
    +
    +        if (partitionType == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() found no partition");
    +            }
    +            Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +            fragments.add(fragment);
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() successful");
    +            }
    +            return fragments;
    +        }
    +
    +        switch (partitionType) {
    +            case DATE: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found DATE partition");
    +                }
    +                int currInterval = intervalNum;
    +
    +                Calendar fragStart = rangeStart;
    +                while (fragStart.before(rangeEnd)) {
    +                    Calendar fragEnd = (Calendar) fragStart.clone();
    +                    switch (intervalType) {
    +                        case DAY:
    +                            fragEnd.add(Calendar.DAY_OF_MONTH, currInterval);
    +                            break;
    +                        case MONTH:
    +                            fragEnd.add(Calendar.MONTH, currInterval);
    +                            break;
    +                        case YEAR:
    +                            fragEnd.add(Calendar.YEAR, currInterval);
    +                            break;
    +                    }
    +                    if (fragEnd.after(rangeEnd))
    +                        fragEnd = (Calendar) rangeEnd.clone();
    +
    +                    // Note that the date is stored in milliseconds
    +                    byte[] msStart = ByteUtil.getBytes(fragStart.getTimeInMillis());
    +                    byte[] msEnd = ByteUtil.getBytes(fragEnd.getTimeInMillis());
    +                    fragmentMetadata = ArrayUtils.addAll(msStart, msEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;
    +                }
    +                break;
    +            }
    +            case INT: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found INT partition");
    +                }
    +                int rangeStart = Integer.parseInt(range[0]);
    +                int rangeEnd = Integer.parseInt(range[1]);
    +                int currInterval = intervalNum;
    +
    +                int fragStart = rangeStart;
    +                while (fragStart < rangeEnd) {
    +                    int fragEnd = fragStart + currInterval;
    +                    if (fragEnd > rangeEnd) {
    +                        fragEnd = rangeEnd;
    +                    }
    +
    +                    byte[] bStart = ByteUtil.getBytes(fragStart);
    +                    byte[] bEnd = ByteUtil.getBytes(fragEnd);
    +                    fragmentMetadata = ArrayUtils.addAll(bStart, bEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;
    +                }
    +                break;
    +            }
    +            case ENUM: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found ENUM partition");
    +                }
    +                for (String frag : range) {
    +                    fragmentMetadata = frag.getBytes();
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +                }
    +                break;
    +            }
    +            default: {
    +                throw new UnsupportedOperationException("getFragments() found a partition of unknown type and failed");
    +            }
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() successful");
    +        }
    +        return fragments;
    +    }
    +
    +    /**
    +     * Insert partition constraints into the prepared SQL query.
    +     * 
    +     * @param inputData pre-validated PXF InputData
    +     * @param sb the SQL query that is prepared for appending WHERE constraints.
    +     * Other SQL statements may be present, but they must be complete. Note that no check is performed to check their "completeness"
    +     */
    +    public static StringBuilder buildFragmenterSql(InputData inputData, StringBuilder sb) {
    --- End diff --
    
    The `StringBuilder sb` may include other WHERE constraints (e.g., partition constraints), thus the result of `buildFragmenterSql()` strongly depends on the contents of `sb`. 
    
    Consider the following example. Let `buildFragmenterSql` return a `String`. In some cases the return value will be like `WHERE id=10 AND ...`, while in other cases ` AND id=10 AND ...`. In the second case, the output has no sense without the initial query, so both the initial query and the result of the function must be present.
    
    Besides, passing a mutable parameter here points to the fact that the existing `WHERE` constraints (present in `StringBuilder sb`) must be complete and that the query will be changed accordingly.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171052327
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    +        return true;
    +    }
    +
    +    /**
    +     * readNextObject() implementation
    +     */
    +    @Override
    +    public OneRow readNextObject() throws Exception {
    +        if (urlReadFetch == null) {
    +            LOG.error("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +            throw new ProtocolException("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +        }
    +
    +        if (bufferRead.isEmpty()) {
    +            // Refill buffer
    +            if (isLastReadFinished) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +
    +            JsonElement response = sendRestRequest(urlReadFetch);
    +            isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +
    +            // Parse 'items'
    +            Iterator<JsonElement> itemsIterator = response.getAsJsonObject().get("items").getAsJsonArray().iterator();
    +            while (itemsIterator.hasNext()) {
    +                if (!bufferRead.add(itemsIterator.next().getAsJsonArray())) {
    +                    LOG.error("readNextObject(): Buffer refill failed (not enough memory in 'bufferRead')");
    +                    throw new OutOfMemoryError("readNextObject(): not enough memory in 'bufferRead'");
    +                }
    +            }
    +
    +            // Check again in case "response" contains no elements
    +            if (bufferRead.isEmpty()) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): Buffer refill failed");
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +        }
    +
    +        return new OneRow(bufferRead.pollFirst());
    +    }
    +
    +    /**
    +     * closeForRead() implementation
    +     */
    +    @Override
    +    public void closeForRead() {
    +        if (urlReadClose != null) {
    +            try {
    +                sendRestRequest(urlReadClose);
    +            }
    +            catch (Exception e) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("closeForRead() Exception: " + e.getClass().getSimpleName());
    +                }
    +            }
    +        }
    +        isLastReadFinished = false;
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Ignite read request finished. URL: '" + urlReadClose + "'");
    +        }
    +    }
    +
    +    /**
    +     * openForWrite() implementation.
    +     * No queries are sent to Ignite by this procedure, so if there are some problems (for example, with connection), they will be revealed only during the execution of 'writeNextObject()'
    +     */
    +    @Override
    +    public boolean openForWrite() throws UserDataException {
    +        // This is a temporary solution. At the moment there is no other way (except for the usage of user-defined parameters) to get the correct name of Ignite table: GPDB inserts extra data into the address, as required by Hadoop.
    +        // Note that if no extra data is present, the 'definedSource' will be left unchanged
    +        String definedSource = inputData.getDataSource();
    +        Pattern pattern = Pattern.compile("/(.*)/[0-9]*-[0-9]*_[0-9]*");
    +        Matcher matcher = pattern.matcher(definedSource);
    +        if (matcher.find()) {
    +            inputData.setDataSource(matcher.group(1));
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("INSERT INTO ");
    +
    +        // Insert the table name
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert the column names
    +        sb.append("(");
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        String fieldDivisor = "";
    +        for (int i = 0; i < columns.size(); i++) {
    +            sb.append(fieldDivisor);
    +            fieldDivisor = ", ";
    +            sb.append(columns.get(i).columnName());
    +        }
    +        sb.append(")");
    +
    +        sb.append(" VALUES ");
    +
    +        queryWrite = sb.toString();
    +        return true;
    +    }
    +
    +    /**
    +     * writeNextObject() implementation
    +     */
    +    @Override
    +    public boolean writeNextObject(OneRow currentRow) throws Exception {
    +        boolean currentRowInBuffer = bufferWrite.add(currentRow);
    +
    +        if (!isWriteActive) {
    +            if (!currentRowInBuffer) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferWrite')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferWrite'");
    --- End diff --
    
    same comment about OOME


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171038185
  
    --- Diff: pxf/pxf-ignite/README.md ---
    @@ -0,0 +1,67 @@
    +# Accessing Ignite database using PXF
    +
    +The PXF Ignite plug-in enables to access the [Ignite database](https://ignite.apache.org/) (both read and write operations are supported) via REST API.
    +
    +
    +# Prerequisites
    +
    +Check the following before using the plug-in:
    +
    +* The Ignite plug-in is installed on all PXF nodes;
    +
    +* The Ignite client is installed and running at the `IGNITE_HOST` (`localhost` by default; see below), and it accepts http queries from the PXF (note that *enabling Ignite REST API does not require changes in Ignite configuration*; see the instruction on how to do that at https://apacheignite.readme.io/docs/rest-api#section-getting-started).
    +
    +
    +# Syntax
    +
    +```
    +CREATE [READABLE] EXTERNAL TABLE <table_name> (
    +    <column_name> <data_type>[, <column_name> <data_type>, ...] | LIKE <other_table>
    +)
    +LOCATION ('pxf://<ignite_table_name>?PROFILE=Ignite[&<extra-parameter>&<extra-parameter>&...]')
    +FORMAT 'CUSTOM' (formatter='pxfwritable_import');
    +```
    +where each `<extra-parameter>` is one of the following:
    +* `IGNITE_HOST=<ignite_host_address_with_port>`. The location of Ignite client node. If not given, `127.0.0.1:8080` is used by default;
    +* `IGNITE_CACHE=<ignite_cache_name>`. The name of Ignite cache to use. If not given, this parameter is not included in queries from PXF to Ignite, thus Ignite default values will be used (at the moment, this is `Default` cache). This option is **case-sensitive**;
    +* `BUFFER_SIZE=<unsigned_int>`. The number of tuples send to (from) Ignite per a response. The same number of tuples is stored in in-plug-in cache. The values `0` and `1` are equal (cache is not used, each tuple is passed in it's own query to Ignite). If not given, `128` is used by default;
    +* `PARTITION_BY=<column>:<column_type>`. See below;
    +* `RANGE=<start_value>:<end_value>`. See below;
    +* `INTERVAL=<value>[:<unit>]`. See below.
    +
    +
    +# Partitioning
    +## Introduction
    +
    +PXF Ignite plugin supports simultaneous access to Ignite database from multiple PXF segments. *Partitioning* should be used in order to perform such operation.
    +
    +If the partitioning is not used, all the data will be retrieved by a single PXF segment.
    +
    +Partitioning in PXF Ignite plug-in works just like in PXF JDBC plug-in.
    +
    +This feature is optional. However, a bug in the `pxf-service` which makes partitioning necessary for any query was fixed only on 17th Jan 2018 in [this commit](https://github.com/apache/incubator-hawq/commit/0d620e431026834dd70c9e0d63edf8bb28b38227), so the older versions of PXF may return an exception if a query does not contain a meaningful `PARTITION_BY` parameter.
    +
    +
    +## Syntax
    +
    +To use partitions, add a set of `<ignite-parameter>`s:
    +```
    +&PARTITION_BY=<column>:<column_type>&RANGE=<start_value>:<end_value>[&INTERVAL=<value>[:<unit>]]
    +```
    +
    +* The `PARTITION_BY` parameter indicates which column to use as the partition column. Only one column can be used as a partition column.
    +    * The `<column>` is the name of a partition column;
    +    * The `<column_type>` is the datatype of a partition column. At the moment, the **supported types** are `INT`, `DATE` and `ENUM`. The `DATE` format is `yyyy-MM-dd`.
    +
    +* The `RANGE` parameter indicates the range of data to be queried. It is left-closed, thus it produces ranges like:
    +    * `[ <start_value> ; <end_value> )`, 
    +    * `... >= start_value AND ... < end_value`;
    --- End diff --
    
    would the logic automatically add  [-infinity:start_value) and [end_value:infinity) ranges ? If not, how will the data that falls into these intervals be retrieved ?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171115576
  
    --- Diff: pxf/pxf-service/src/main/resources/pxf-profiles-default.xml ---
    @@ -193,4 +193,13 @@ under the License.
                 <resolver>org.apache.hawq.pxf.plugins.hdfs.ParquetResolver</resolver>
             </plugins>
         </profile>
    +    <profile>
    +        <name>Ignite</name>
    +        <description>A profile to read and write data from/to Ignite</description>
    --- End diff --
    
    should refer to it as Apache Ignite ?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171073052
  
    --- Diff: pxf/pxf-ignite/README.md ---
    @@ -0,0 +1,67 @@
    +# Accessing Ignite database using PXF
    +
    +The PXF Ignite plug-in enables to access the [Ignite database](https://ignite.apache.org/) (both read and write operations are supported) via REST API.
    +
    +
    +# Prerequisites
    +
    +Check the following before using the plug-in:
    +
    +* The Ignite plug-in is installed on all PXF nodes;
    +
    +* The Ignite client is installed and running at the `IGNITE_HOST` (`localhost` by default; see below), and it accepts http queries from the PXF (note that *enabling Ignite REST API does not require changes in Ignite configuration*; see the instruction on how to do that at https://apacheignite.readme.io/docs/rest-api#section-getting-started).
    +
    +
    +# Syntax
    +
    +```
    +CREATE [READABLE] EXTERNAL TABLE <table_name> (
    +    <column_name> <data_type>[, <column_name> <data_type>, ...] | LIKE <other_table>
    +)
    +LOCATION ('pxf://<ignite_table_name>?PROFILE=Ignite[&<extra-parameter>&<extra-parameter>&...]')
    +FORMAT 'CUSTOM' (formatter='pxfwritable_import');
    +```
    +where each `<extra-parameter>` is one of the following:
    +* `IGNITE_HOST=<ignite_host_address_with_port>`. The location of Ignite client node. If not given, `127.0.0.1:8080` is used by default;
    +* `IGNITE_CACHE=<ignite_cache_name>`. The name of Ignite cache to use. If not given, this parameter is not included in queries from PXF to Ignite, thus Ignite default values will be used (at the moment, this is `Default` cache). This option is **case-sensitive**;
    +* `BUFFER_SIZE=<unsigned_int>`. The number of tuples send to (from) Ignite per a response. The same number of tuples is stored in in-plug-in cache. The values `0` and `1` are equal (cache is not used, each tuple is passed in it's own query to Ignite). If not given, `128` is used by default;
    +* `PARTITION_BY=<column>:<column_type>`. See below;
    +* `RANGE=<start_value>:<end_value>`. See below;
    +* `INTERVAL=<value>[:<unit>]`. See below.
    +
    +
    +# Partitioning
    +## Introduction
    +
    +PXF Ignite plugin supports simultaneous access to Ignite database from multiple PXF segments. *Partitioning* should be used in order to perform such operation.
    +
    +If the partitioning is not used, all the data will be retrieved by a single PXF segment.
    +
    +Partitioning in PXF Ignite plug-in works just like in PXF JDBC plug-in.
    +
    +This feature is optional. However, a bug in the `pxf-service` which makes partitioning necessary for any query was fixed only on 17th Jan 2018 in [this commit](https://github.com/apache/incubator-hawq/commit/0d620e431026834dd70c9e0d63edf8bb28b38227), so the older versions of PXF may return an exception if a query does not contain a meaningful `PARTITION_BY` parameter.
    +
    +
    +## Syntax
    +
    +To use partitions, add a set of `<ignite-parameter>`s:
    +```
    +&PARTITION_BY=<column>:<column_type>&RANGE=<start_value>:<end_value>[&INTERVAL=<value>[:<unit>]]
    +```
    +
    +* The `PARTITION_BY` parameter indicates which column to use as the partition column. Only one column can be used as a partition column.
    +    * The `<column>` is the name of a partition column;
    +    * The `<column_type>` is the datatype of a partition column. At the moment, the **supported types** are `INT`, `DATE` and `ENUM`. The `DATE` format is `yyyy-MM-dd`.
    +
    +* The `RANGE` parameter indicates the range of data to be queried. It is left-closed, thus it produces ranges like:
    +    * `[ <start_value> ; <end_value> )`, 
    +    * `... >= start_value AND ... < end_value`;
    +
    +* The `INTERVAL` parameter is **required** for `INT` and `DATE` partitions. This parameter is ignored if `<column_type>` is `ENUM`.
    +    * The `<value>` is the size of each fragment (the last one may be smaller). Note that by default PXF does not support more than 100 fragments;
    +    * The `<unit>` **must** be provided if `<column_type>` is `DATE`. At the moment, only `year`, `month` and `day` are supported. This parameter is ignored in case of any other `<column_type>`.
    +
    +Example partitions:
    +* `&PARTITION_BY=id:int&RANGE=42:142&INTERVAL=1`
    +* `&PARTITION_BY=createdate:date&RANGE=2008-01-01:2010-01-01&INTERVAL=1:month`
    +* `&PARTITION_BY=grade:enum&RANGE=excellent:good:general:bad`
    --- End diff --
    
    looks like for `ENUM` type, the `RANGE` is a list of the supported values. Could you please update the documentation for `RANGE` with that syntax?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171055350
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    +        return true;
    +    }
    +
    +    /**
    +     * readNextObject() implementation
    +     */
    +    @Override
    +    public OneRow readNextObject() throws Exception {
    +        if (urlReadFetch == null) {
    +            LOG.error("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +            throw new ProtocolException("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +        }
    +
    +        if (bufferRead.isEmpty()) {
    +            // Refill buffer
    +            if (isLastReadFinished) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +
    +            JsonElement response = sendRestRequest(urlReadFetch);
    +            isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +
    +            // Parse 'items'
    +            Iterator<JsonElement> itemsIterator = response.getAsJsonObject().get("items").getAsJsonArray().iterator();
    +            while (itemsIterator.hasNext()) {
    +                if (!bufferRead.add(itemsIterator.next().getAsJsonArray())) {
    +                    LOG.error("readNextObject(): Buffer refill failed (not enough memory in 'bufferRead')");
    +                    throw new OutOfMemoryError("readNextObject(): not enough memory in 'bufferRead'");
    +                }
    +            }
    +
    +            // Check again in case "response" contains no elements
    +            if (bufferRead.isEmpty()) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): Buffer refill failed");
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +        }
    +
    +        return new OneRow(bufferRead.pollFirst());
    +    }
    +
    +    /**
    +     * closeForRead() implementation
    +     */
    +    @Override
    +    public void closeForRead() {
    +        if (urlReadClose != null) {
    +            try {
    +                sendRestRequest(urlReadClose);
    +            }
    +            catch (Exception e) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("closeForRead() Exception: " + e.getClass().getSimpleName());
    +                }
    +            }
    +        }
    +        isLastReadFinished = false;
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Ignite read request finished. URL: '" + urlReadClose + "'");
    +        }
    +    }
    +
    +    /**
    +     * openForWrite() implementation.
    +     * No queries are sent to Ignite by this procedure, so if there are some problems (for example, with connection), they will be revealed only during the execution of 'writeNextObject()'
    +     */
    +    @Override
    +    public boolean openForWrite() throws UserDataException {
    +        // This is a temporary solution. At the moment there is no other way (except for the usage of user-defined parameters) to get the correct name of Ignite table: GPDB inserts extra data into the address, as required by Hadoop.
    +        // Note that if no extra data is present, the 'definedSource' will be left unchanged
    +        String definedSource = inputData.getDataSource();
    +        Pattern pattern = Pattern.compile("/(.*)/[0-9]*-[0-9]*_[0-9]*");
    +        Matcher matcher = pattern.matcher(definedSource);
    +        if (matcher.find()) {
    +            inputData.setDataSource(matcher.group(1));
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("INSERT INTO ");
    +
    +        // Insert the table name
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert the column names
    +        sb.append("(");
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        String fieldDivisor = "";
    +        for (int i = 0; i < columns.size(); i++) {
    +            sb.append(fieldDivisor);
    +            fieldDivisor = ", ";
    +            sb.append(columns.get(i).columnName());
    +        }
    +        sb.append(")");
    +
    +        sb.append(" VALUES ");
    +
    +        queryWrite = sb.toString();
    +        return true;
    +    }
    +
    +    /**
    +     * writeNextObject() implementation
    +     */
    +    @Override
    +    public boolean writeNextObject(OneRow currentRow) throws Exception {
    +        boolean currentRowInBuffer = bufferWrite.add(currentRow);
    +
    +        if (!isWriteActive) {
    +            if (!currentRowInBuffer) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferWrite')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferWrite'");
    +            }
    +            LOG.info("Ignite write request. Query: '" + queryWrite + "'");
    +            sendInsertRestRequest(queryWrite);
    +            bufferWrite.removeFirst();
    +            isWriteActive = true;
    +            return true;
    +        }
    +
    +        if ((bufferWrite.size() >= bufferSize) || (!currentRowInBuffer)) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +
    +        if (!currentRowInBuffer) {
    +            if (!bufferWrite.add(currentRow)) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferSend')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferSend'");
    +            }
    +        }
    +        
    +        return true;
    +    }
    +
    +    /**
    +     * closeForWrite() implementation
    +     */
    +    @Override
    +    public void closeForWrite() throws Exception {
    +        if (!bufferWrite.isEmpty()) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +        if (isWriteActive) {
    +            // At this point, the request must have finished successfully; otherwise an exception would be thrown before
    +            LOG.info("Ignite write request finished successfully. Query: '" + queryWrite + "'");
    +        }
    +        isWriteActive = false;
    +    }
    +
    +    /**
    +     * Build HTTP GET query for Ignite REST API with command 'qryfldexe'
    +     * 
    +     * @param querySql SQL query with filter constants. The constants must replaced by "?" if 'filterConstants' is not given
    +     * @param filterConstants
    +     * 
    +     * @return Prepared HTTP query. The query will be properly encoded with {@link java.net.URLEncoder}
    +     * 
    +     * @throws UnsupportedEncodingException from {@link java.net.URLEncoder.encode()}
    +     */
    +    private String buildQueryFldexe(String querySql, ArrayList<String> filterConstants) throws UnsupportedEncodingException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("http://");
    +        sb.append(igniteHost);
    +        sb.append("/ignite");
    +        sb.append("?");
    +        sb.append("cmd=qryfldexe");
    +        sb.append("&");
    +        sb.append("pageSize=0");
    +        sb.append("&");
    +        if (cacheName != null) {
    +            sb.append("cacheName=");
    +            // Note that Ignite supports only "good" cache names (those that should be left unchanged by the URLEncoder.encode())
    +            sb.append(URLEncoder.encode(cacheName, "UTF-8"));
    +            sb.append("&");
    +        }
    +        int counter = 1;
    +        if (filterConstants != null) {
    +            for (String constant : filterConstants) {
    +                sb.append("arg");
    +                sb.append(counter);
    +                sb.append("=");
    +                sb.append(URLEncoder.encode(constant, "UTF-8"));
    +                sb.append("&");
    +                counter += 1;
    +            }
    +        }
    +        sb.append("qry=");
    +        sb.append(URLEncoder.encode(querySql, "UTF-8"));
    +
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Build HTTP GET query for Ignite REST API with command 'qryfetch'
    +     * This query is used to retrieve data after the 'qryfldexe' command started
    +     * 
    +     * @param queryId ID of the query assigned by Ignite when the query started
    +     *
    +     * @return Prepared HTTP query
    +     */
    +    private String buildQueryFetch(int queryId) {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("http://");
    +        sb.append(igniteHost);
    +        sb.append("/ignite");
    +        sb.append("?");
    +        sb.append("cmd=qryfetch");
    +        sb.append("&");
    +        sb.append("pageSize=");
    +        sb.append(bufferSize);
    +        sb.append("&");
    +        sb.append("qryId=");
    +        sb.append(queryId);
    +
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Build HTTP GET query for Ignite REST API with command 'qrycls'
    +     * This query is used to close query resources on Ignite side
    +     * 
    +     * @param queryId ID of the query assigned by Ignite when the query started
    +     *
    +     * @return Prepared HTTP query
    +     */
    +    private String buildQueryCls(int queryId) {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("http://");
    +        sb.append(igniteHost);
    +        sb.append("/ignite");
    +        sb.append("?");
    +        sb.append("cmd=qrycls");
    +        sb.append("&");
    +        sb.append("qryId=");
    +        sb.append(queryId);
    +
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Send a REST request to the Ignite server
    +     * 
    +     * @param query A prepared and properly encoded HTTP GET request
    +     * 
    +     * @return "response" field from the received JSON object
    +     * (See Ignite REST API documentation for details)
    +     * 
    +     * @throws ProtocolException if Ignite reports error in it's JSON response
    +     * @throws MalformedURLException if URL is malformed
    +     * @throws IOException in case of connection failure
    +     */
    +    private JsonElement sendRestRequest(String query) throws ProtocolException, MalformedURLException, IOException {
    +        // Create URL object
    +        URL url;
    +        try {
    +            url = new URL(query);
    +        }
    +        catch (MalformedURLException e) {
    +            LOG.error("sendRestRequest(): Failed (malformed URL). URL is '" + query + "'");
    +            throw e;
    +        }
    +
    +        // Connect to the Ignite server, send query and get raw response
    +        String responseRaw = null;
    +        try {
    +            StringBuilder sb = new StringBuilder();
    +            BufferedReader reader = new BufferedReader(new InputStreamReader(url.openStream()));
    --- End diff --
    
    you might consider using some kind of HTTP Client library that caches connections for better performance, since you're always connecting to the same Ignite server.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171440009
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,411 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +
    +/**
    + * PXF-Ignite fragmenter class
    + * 
    + * This fragmenter works just like the one in PXF JDBC plugin
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    /**
    +     * Class constructor
    +     * 
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inputData) throws UserDataException {
    +        super(inputData);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inputData.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inputData.getUserProperty("PARTITION_BY").split(":");
    +            partitionType = PartitionType.typeOf(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' is invalid. The pattern is 'column_name:DATE|INT|ENUM'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inputData.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' is invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inputData.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.typeOf(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    +                }
    +            } 
    +            else if (partitionType != PartitionType.ENUM) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be specified along with 'PARTITION_BY'");
    +            }
    +            if (intervalNum < 1) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be at least 1. The actual is '" + intervalNum + "'");
    +            }
    +        } 
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'INTERVAL' invalid. The pattern is 'interval_num[:interval_unit]'");
    +        }
    +
    +        //parse date values
    +        try {
    +            if (partitionType == PartitionType.DATE) {
    +                SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd");
    +                rangeStart = Calendar.getInstance();
    +                rangeStart.setTime(df.parse(range[0]));
    +                rangeEnd = Calendar.getInstance();
    +                rangeEnd.setTime(df.parse(range[1]));
    +            }
    +        } catch (ParseException e) {
    +            throw new UserDataException("The parameter 'RANGE' has invalid date format. Expected format is 'yyyy-MM-dd'");
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful; some partition used");
    +        }
    +    }
    +
    +    /**
    +     * Returns statistics for the Ignite table. This is not implemented in the current version
    +     * @throws UnsupportedOperationException
    +     */
    +    @Override
    +    public FragmentsStats getFragmentsStats() throws UnsupportedOperationException {
    +        throw new UnsupportedOperationException("ANALYZE for Ignite plugin is not supported");
    +    }
    +
    +    /**
    +     * Returns list of fragments for Ignite table queries
    +     *
    +     * @throws UnsupportedOperationException if a partition of unknown type was found
    +     * 
    +     * @return a list of fragments
    +     */
    +    @Override
    +    public List<Fragment> getFragments() throws UnsupportedOperationException {
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() called; dataSource is '" + inputData.getDataSource() + "'");
    +        }
    +
    +        byte[] fragmentMetadata = null;
    +        byte[] fragmentUserdata = null;
    +
    +        if (partitionType == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() found no partition");
    +            }
    +            Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +            fragments.add(fragment);
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() successful");
    +            }
    +            return fragments;
    +        }
    +
    +        switch (partitionType) {
    +            case DATE: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found DATE partition");
    +                }
    +                int currInterval = intervalNum;
    +
    +                Calendar fragStart = rangeStart;
    +                while (fragStart.before(rangeEnd)) {
    +                    Calendar fragEnd = (Calendar) fragStart.clone();
    +                    switch (intervalType) {
    +                        case DAY:
    +                            fragEnd.add(Calendar.DAY_OF_MONTH, currInterval);
    +                            break;
    +                        case MONTH:
    +                            fragEnd.add(Calendar.MONTH, currInterval);
    +                            break;
    +                        case YEAR:
    +                            fragEnd.add(Calendar.YEAR, currInterval);
    +                            break;
    +                    }
    +                    if (fragEnd.after(rangeEnd))
    +                        fragEnd = (Calendar) rangeEnd.clone();
    +
    +                    // Note that the date is stored in milliseconds
    +                    byte[] msStart = ByteUtil.getBytes(fragStart.getTimeInMillis());
    +                    byte[] msEnd = ByteUtil.getBytes(fragEnd.getTimeInMillis());
    +                    fragmentMetadata = ArrayUtils.addAll(msStart, msEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;
    +                }
    +                break;
    +            }
    +            case INT: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found INT partition");
    +                }
    +                int rangeStart = Integer.parseInt(range[0]);
    +                int rangeEnd = Integer.parseInt(range[1]);
    +                int currInterval = intervalNum;
    +
    +                int fragStart = rangeStart;
    +                while (fragStart < rangeEnd) {
    +                    int fragEnd = fragStart + currInterval;
    +                    if (fragEnd > rangeEnd) {
    +                        fragEnd = rangeEnd;
    +                    }
    +
    +                    byte[] bStart = ByteUtil.getBytes(fragStart);
    +                    byte[] bEnd = ByteUtil.getBytes(fragEnd);
    +                    fragmentMetadata = ArrayUtils.addAll(bStart, bEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;
    +                }
    +                break;
    +            }
    +            case ENUM: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found ENUM partition");
    +                }
    +                for (String frag : range) {
    +                    fragmentMetadata = frag.getBytes();
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +                }
    +                break;
    +            }
    +            default: {
    +                throw new UnsupportedOperationException("getFragments() found a partition of unknown type and failed");
    +            }
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() successful");
    +        }
    +        return fragments;
    +    }
    +
    +    /**
    +     * Insert partition constraints into the prepared SQL query.
    +     * 
    +     * @param inputData pre-validated PXF InputData
    +     * @param sb the SQL query that is prepared for appending WHERE constraints.
    +     * Other SQL statements may be present, but they must be complete. Note that no check is performed to check their "completeness"
    +     */
    +    public static StringBuilder buildFragmenterSql(InputData inputData, StringBuilder sb) {
    --- End diff --
    
    Do we really want to mutate input parameter and then return it?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171052811
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    +        return true;
    +    }
    +
    +    /**
    +     * readNextObject() implementation
    +     */
    +    @Override
    +    public OneRow readNextObject() throws Exception {
    +        if (urlReadFetch == null) {
    +            LOG.error("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +            throw new ProtocolException("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +        }
    +
    +        if (bufferRead.isEmpty()) {
    +            // Refill buffer
    +            if (isLastReadFinished) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +
    +            JsonElement response = sendRestRequest(urlReadFetch);
    +            isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +
    +            // Parse 'items'
    +            Iterator<JsonElement> itemsIterator = response.getAsJsonObject().get("items").getAsJsonArray().iterator();
    +            while (itemsIterator.hasNext()) {
    +                if (!bufferRead.add(itemsIterator.next().getAsJsonArray())) {
    +                    LOG.error("readNextObject(): Buffer refill failed (not enough memory in 'bufferRead')");
    +                    throw new OutOfMemoryError("readNextObject(): not enough memory in 'bufferRead'");
    +                }
    +            }
    +
    +            // Check again in case "response" contains no elements
    +            if (bufferRead.isEmpty()) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): Buffer refill failed");
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +        }
    +
    +        return new OneRow(bufferRead.pollFirst());
    +    }
    +
    +    /**
    +     * closeForRead() implementation
    +     */
    +    @Override
    +    public void closeForRead() {
    +        if (urlReadClose != null) {
    +            try {
    +                sendRestRequest(urlReadClose);
    +            }
    +            catch (Exception e) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("closeForRead() Exception: " + e.getClass().getSimpleName());
    +                }
    +            }
    +        }
    +        isLastReadFinished = false;
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Ignite read request finished. URL: '" + urlReadClose + "'");
    +        }
    +    }
    +
    +    /**
    +     * openForWrite() implementation.
    +     * No queries are sent to Ignite by this procedure, so if there are some problems (for example, with connection), they will be revealed only during the execution of 'writeNextObject()'
    +     */
    +    @Override
    +    public boolean openForWrite() throws UserDataException {
    +        // This is a temporary solution. At the moment there is no other way (except for the usage of user-defined parameters) to get the correct name of Ignite table: GPDB inserts extra data into the address, as required by Hadoop.
    +        // Note that if no extra data is present, the 'definedSource' will be left unchanged
    +        String definedSource = inputData.getDataSource();
    +        Pattern pattern = Pattern.compile("/(.*)/[0-9]*-[0-9]*_[0-9]*");
    +        Matcher matcher = pattern.matcher(definedSource);
    +        if (matcher.find()) {
    +            inputData.setDataSource(matcher.group(1));
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("INSERT INTO ");
    +
    +        // Insert the table name
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert the column names
    +        sb.append("(");
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        String fieldDivisor = "";
    +        for (int i = 0; i < columns.size(); i++) {
    +            sb.append(fieldDivisor);
    +            fieldDivisor = ", ";
    +            sb.append(columns.get(i).columnName());
    +        }
    +        sb.append(")");
    +
    +        sb.append(" VALUES ");
    +
    +        queryWrite = sb.toString();
    +        return true;
    +    }
    +
    +    /**
    +     * writeNextObject() implementation
    +     */
    +    @Override
    +    public boolean writeNextObject(OneRow currentRow) throws Exception {
    +        boolean currentRowInBuffer = bufferWrite.add(currentRow);
    +
    +        if (!isWriteActive) {
    +            if (!currentRowInBuffer) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferWrite')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferWrite'");
    +            }
    +            LOG.info("Ignite write request. Query: '" + queryWrite + "'");
    +            sendInsertRestRequest(queryWrite);
    +            bufferWrite.removeFirst();
    +            isWriteActive = true;
    +            return true;
    +        }
    +
    +        if ((bufferWrite.size() >= bufferSize) || (!currentRowInBuffer)) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +
    +        if (!currentRowInBuffer) {
    +            if (!bufferWrite.add(currentRow)) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferSend')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferSend'");
    --- End diff --
    
    same


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171114273
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePlugin.java ---
    @@ -0,0 +1,90 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.utilities.Plugin;
    +
    +/**
    + * This class resolves the jdbc connection parameter and manages the opening and closing of the jdbc connection.
    --- End diff --
    
    copy & paste from JDBC ?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171144921
  
    --- Diff: pxf/pxf-ignite/src/test/java/org/apache/hawq/pxf/plugins/ignite/SqlBuilderTest.java ---
    @@ -0,0 +1,173 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hawq.pxf.api.*;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.io.DataType;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +/**
    + * Validate SQL string generated by the {@link IgnitePartitionFragmenter#buildFragmenterSql} method
    + * and the {@link WhereSQLBuilder#buildWhereSQL} method.
    + */
    +public class SqlBuilderTest {
    +    private static final Log LOG = LogFactory.getLog(SqlBuilderTest.class);
    +    static final String ORIGINAL_SQL = "SELECT * FROM sales";
    +    InputData inputData;
    +
    +    @Before
    --- End diff --
    
    are the before/after necessary?


---

[GitHub] incubator-hawq issue #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344
  
    @lisakowen, 
    * Ubuntu 16.04; CentOS 7.4
    * Gradle 2.10, Visual Studio Code 1.20.1, IntelliJ Idea 2017.3
    * Key sources (the most useful first): http://hawq.incubator.apache.org/docs/userguide/2.1.0.0-incubating/pxf/PXFExternalTableandAPIReference.html (this article is far more useful than anything else I found and used); PXF JDBC plug-in code; PXF API code
    * Accessor. The key details on how to implement other classes are already present in other plugins
    * I am going to expand the README to add information on `INSERT` queries and perfomance test results. Other documentation is mostly in the code itself. If adding some extra docs would be really helpful, please let me know.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171132308
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,384 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +/**
    + * Fragmenter class for Ignite data resources.
    + * @note At the moment this fragmenter works just like the JDBC's one
    + *
    + * Aside from implementing PXF {@link Fragmenter},
    + * this class also transforms input data path (given by user) by adding the
    + * information on splitting the request into a few requests for different regions.
    + * <br>
    + * The parameter Patterns
    + * <br>
    + * There are three  parameters, the format is as follows:
    + * <br>
    + * <pre>
    + * <code>PARTITION_BY=column_name:column_type&amp;RANGE=start_value[:end_value]&amp;INTERVAL=interval_num[:interval_unit]</code>
    + * </pre>
    + * The <code>PARTITION_BY</code> parameter can be split by colon(':'),the <code>column_type</code> current supported : <code>date,int,enum</code> .
    + * The Date format is 'yyyy-MM-dd'. <br>
    + * The <code>RANGE</code> parameter can be split by colon(':') ,used to identify the starting range of each fragment.
    + * The range is left-closed, ie:<code> '&gt;= start_value AND &lt; end_value' </code>.If the <code>column_type</code> is <code>int</code>,
    + * the <code>end_value</code> can be empty. If the <code>column_type</code>is <code>enum</code>,the parameter <code>RANGE</code> can be empty. <br>
    + * The <code>INTERVAL</code> parameter can be split by colon(':'), indicate the interval value of one fragment.
    + * When <code>column_type</code> is <code>date</code>,this parameter must be split by colon, and <code>interval_unit</code> can be <code>year,month,day</code>.
    + * When <code>column_type</code> is <code>int</code>, the <code>interval_unit</code> can be empty.
    + * When <code>column_type</code> is <code>enum</code>,the <code>INTERVAL</code> parameter can be empty.
    + * <br>
    + * <p>
    + * The syntax examples is:
    + * <br>
    + * <code>PARTITION_BY=createdate:date&amp;RANGE=2008-01-01:2010-01-01&amp;INTERVAL=1:month'</code>
    + * <br>
    + * <code>PARTITION_BY=year:int&amp;RANGE=2008:2010&amp;INTERVAL=1</code>
    + * <br>
    + * <code>PARTITION_BY=grade:enum&amp;RANGE=excellent:good:general:bad</code>
    + * </p>
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    Log LOG = LogFactory.getLog(IgnitePartitionFragmenter.class);
    +
    +    String[] partitionBy = null;
    +    String[] range = null;
    +    String[] interval = null;
    +    PartitionType partitionType = null;
    +    String partitionColumn = null;
    +    IntervalType intervalType = null;
    +    int intervalNum = 1;
    +
    +    //when partitionType is DATE,it is valid
    +    Calendar rangeStart = null;
    +    Calendar rangeEnd = null;
    +
    +
    +    enum PartitionType {
    +        DATE,
    +        INT,
    +        ENUM;
    +
    +        public static PartitionType getType(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    enum IntervalType {
    +        DAY,
    +        MONTH,
    +        YEAR;
    +
    +        public static IntervalType type(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    /**
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inConf) throws UserDataException {
    +        super(inConf);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inConf.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inConf.getUserProperty("PARTITION_BY").split(":");
    +            partitionColumn = partitionBy[0];
    +            partitionType = PartitionType.getType(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' invalid, the pattern is 'column_name:date|int|enum'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inConf.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inConf.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.type(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    +                }
    +            } 
    +            else if (partitionType != PartitionType.ENUM) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be specified along with 'PARTITION_BY'");
    +            }
    +            if (intervalNum < 1) {
    +                throw new UserDataException("The parameter 'INTERVAL' must > 1, but actual is '" + intervalNum + "'");
    +            }
    +        } 
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'INTERVAL' invalid, the pattern is 'interval_num[:interval_unit]'");
    +        }
    +
    +        //parse any date values
    +        try {
    +            if (partitionType == PartitionType.DATE) {
    +                SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd");
    +                rangeStart = Calendar.getInstance();
    +                rangeStart.setTime(df.parse(range[0]));
    +                rangeEnd = Calendar.getInstance();
    +                rangeEnd.setTime(df.parse(range[1]));
    +            }
    +        } catch (ParseException e) {
    +            throw new UserDataException("The parameter 'RANGE' has invalid date format. Expected format is 'YYYY-MM-DD'");
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful; some partition used");
    +        }
    +    }
    +
    +    /**
    +     * Returns statistics for  Ignite table. Currently it's not implemented.
    +     * @throws UnsupportedOperationException
    +     */
    +    @Override
    +    public FragmentsStats getFragmentsStats() throws UnsupportedOperationException {
    +        throw new UnsupportedOperationException("ANALYZE for Ignite plugin is not supported");
    +    }
    +
    +    /**
    +     * Returns list of fragments for Ignite table queries
    +     *
    +     * @return a list of fragments
    +     * @throws UnknownHostException from java.net.InetAddress.getLocalHost().getHostAddress()
    +     */
    +    @Override
    +    public List<Fragment> getFragments() throws UnknownHostException {
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() called; dataSource is '" + inputData.getDataSource() + "'");
    +        }
    +
    +        // Always use 'localhost' as a replica holder
    +        // The replica holder is a *PXF* host which can process the requested query
    +        // Note that this is not an Ignite host
    +        String[] replicaHostAddressWrapped = new String[]{InetAddress.getLocalHost().getHostAddress()};
    +        byte[] fragmentMetadata = null;
    +        byte[] fragmentUserdata = null;
    +
    +        if (partitionType == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() found no partition");
    +            }
    +            Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +            fragments.add(fragment);
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() successful. Total fragments: " + fragments.size());
    +            }
    +            return fragments;
    +        }
    +
    +        switch (partitionType) {
    +            case DATE: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found DATE partition");
    +                }
    +                int currInterval = intervalNum;
    +
    +                Calendar fragStart = rangeStart;
    +                while (fragStart.before(rangeEnd)) {
    +                    Calendar fragEnd = (Calendar) fragStart.clone();
    +                    switch (intervalType) {
    +                        case DAY:
    +                            fragEnd.add(Calendar.DAY_OF_MONTH, currInterval);
    +                            break;
    +                        case MONTH:
    +                            fragEnd.add(Calendar.MONTH, currInterval);
    +                            break;
    +                        case YEAR:
    +                            fragEnd.add(Calendar.YEAR, currInterval);
    +                            break;
    +                    }
    +                    if (fragEnd.after(rangeEnd))
    +                        fragEnd = (Calendar) rangeEnd.clone();
    +
    +                    // Build Fragment.metadata: convert the date to a millisecond, then get bytes.
    +                    byte[] msStart = ByteUtil.getBytes(fragStart.getTimeInMillis());
    +                    byte[] msEnd = ByteUtil.getBytes(fragEnd.getTimeInMillis());
    +                    fragmentMetadata = ArrayUtils.addAll(msStart, msEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;
    +                }
    +                break;
    +            }
    +            case INT: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found INT partition");
    +                }
    +                int rangeStart = Integer.parseInt(range[0]);
    +                int rangeEnd = Integer.parseInt(range[1]);
    +                int currInterval = intervalNum;
    +
    +                // Validate : curr_interval > 0
    +                int fragStart = rangeStart;
    +                while (fragStart < rangeEnd) {
    +                    int fragEnd = fragStart + currInterval;
    +                    if (fragEnd > rangeEnd) fragEnd = rangeEnd;
    +
    +                    byte[] bStart = ByteUtil.getBytes(fragStart);
    +                    byte[] bEnd = ByteUtil.getBytes(fragEnd);
    +                    fragmentMetadata = ArrayUtils.addAll(bStart, bEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;// + 1;
    +                }
    +                break;
    +            }
    +            case ENUM:
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found ENUM partition");
    +                }
    +                for (String frag : range) {
    +                    fragmentMetadata = frag.getBytes();
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +                }
    +                break;
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() successful. Total fragments: " + fragments.size());
    +        }
    +        return fragments;
    +    }
    +
    +    public StringBuilder buildFragmenterSql(StringBuilder sb) {
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("buildFragmenterSql() called");
    +        }
    +
    +        byte[] meta = inputData.getFragmentMetadata();
    +        if (meta == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Fragment metadata is null; No partition constraints added");
    +            }
    +            return sb;
    +        }
    +
    +        if (!sb.toString().contains("WHERE")) {
    +            sb.append(" WHERE 1=1 ");
    +        }
    +        sb.append(" AND ");
    +
    +        switch (partitionType) {
    --- End diff --
    
    Not sure if we need to initialize the Fragmenter from the Accessor. If we only use the partitionType and partitionColumn here, we could get them directly from the input data (maybe with a static helper function in the Fragmenter), and make this function static. As it is, we run a lot of validations that are not really necessary at the Accessor stage.


---

[GitHub] incubator-hawq issue #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344
  
    @leskin-in - i am writing pxf developer docs.  i have a few questions if you are willing to share:
    
    - on what OS did you develop the plug-in?
    - what build tool/IDE did you use?
    - where did you find reference material for your work?  HAWQ PXF wiki?  HAWQ docs?  code? - which was the most relevant/helpful?
    - what plug-in (fragmenter, accessor, resolver) did you implement first?
    - do you have any design/specification documents for the ignite work that you would be willing to share?
    
    thank you!


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171081924
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    +        return true;
    +    }
    +
    +    /**
    +     * readNextObject() implementation
    +     */
    +    @Override
    +    public OneRow readNextObject() throws Exception {
    +        if (urlReadFetch == null) {
    +            LOG.error("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +            throw new ProtocolException("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +        }
    +
    +        if (bufferRead.isEmpty()) {
    +            // Refill buffer
    +            if (isLastReadFinished) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +
    +            JsonElement response = sendRestRequest(urlReadFetch);
    +            isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +
    +            // Parse 'items'
    +            Iterator<JsonElement> itemsIterator = response.getAsJsonObject().get("items").getAsJsonArray().iterator();
    +            while (itemsIterator.hasNext()) {
    +                if (!bufferRead.add(itemsIterator.next().getAsJsonArray())) {
    +                    LOG.error("readNextObject(): Buffer refill failed (not enough memory in 'bufferRead')");
    +                    throw new OutOfMemoryError("readNextObject(): not enough memory in 'bufferRead'");
    +                }
    +            }
    +
    +            // Check again in case "response" contains no elements
    +            if (bufferRead.isEmpty()) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): Buffer refill failed");
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +        }
    +
    +        return new OneRow(bufferRead.pollFirst());
    +    }
    +
    +    /**
    +     * closeForRead() implementation
    +     */
    +    @Override
    +    public void closeForRead() {
    +        if (urlReadClose != null) {
    +            try {
    +                sendRestRequest(urlReadClose);
    +            }
    +            catch (Exception e) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("closeForRead() Exception: " + e.getClass().getSimpleName());
    +                }
    +            }
    +        }
    +        isLastReadFinished = false;
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Ignite read request finished. URL: '" + urlReadClose + "'");
    +        }
    +    }
    +
    +    /**
    +     * openForWrite() implementation.
    +     * No queries are sent to Ignite by this procedure, so if there are some problems (for example, with connection), they will be revealed only during the execution of 'writeNextObject()'
    +     */
    +    @Override
    +    public boolean openForWrite() throws UserDataException {
    +        // This is a temporary solution. At the moment there is no other way (except for the usage of user-defined parameters) to get the correct name of Ignite table: GPDB inserts extra data into the address, as required by Hadoop.
    +        // Note that if no extra data is present, the 'definedSource' will be left unchanged
    +        String definedSource = inputData.getDataSource();
    +        Pattern pattern = Pattern.compile("/(.*)/[0-9]*-[0-9]*_[0-9]*");
    +        Matcher matcher = pattern.matcher(definedSource);
    +        if (matcher.find()) {
    +            inputData.setDataSource(matcher.group(1));
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("INSERT INTO ");
    +
    +        // Insert the table name
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert the column names
    +        sb.append("(");
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        String fieldDivisor = "";
    +        for (int i = 0; i < columns.size(); i++) {
    +            sb.append(fieldDivisor);
    +            fieldDivisor = ", ";
    +            sb.append(columns.get(i).columnName());
    +        }
    +        sb.append(")");
    +
    +        sb.append(" VALUES ");
    +
    +        queryWrite = sb.toString();
    +        return true;
    +    }
    +
    +    /**
    +     * writeNextObject() implementation
    +     */
    +    @Override
    +    public boolean writeNextObject(OneRow currentRow) throws Exception {
    +        boolean currentRowInBuffer = bufferWrite.add(currentRow);
    +
    +        if (!isWriteActive) {
    +            if (!currentRowInBuffer) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferWrite')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferWrite'");
    +            }
    +            LOG.info("Ignite write request. Query: '" + queryWrite + "'");
    +            sendInsertRestRequest(queryWrite);
    +            bufferWrite.removeFirst();
    +            isWriteActive = true;
    +            return true;
    +        }
    +
    +        if ((bufferWrite.size() >= bufferSize) || (!currentRowInBuffer)) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +
    +        if (!currentRowInBuffer) {
    +            if (!bufferWrite.add(currentRow)) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferSend')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferSend'");
    +            }
    +        }
    +        
    +        return true;
    +    }
    +
    +    /**
    +     * closeForWrite() implementation
    +     */
    +    @Override
    +    public void closeForWrite() throws Exception {
    +        if (!bufferWrite.isEmpty()) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +        if (isWriteActive) {
    +            // At this point, the request must have finished successfully; otherwise an exception would be thrown before
    +            LOG.info("Ignite write request finished successfully. Query: '" + queryWrite + "'");
    +        }
    +        isWriteActive = false;
    +    }
    +
    +    /**
    +     * Build HTTP GET query for Ignite REST API with command 'qryfldexe'
    +     * 
    +     * @param querySql SQL query with filter constants. The constants must replaced by "?" if 'filterConstants' is not given
    +     * @param filterConstants
    +     * 
    +     * @return Prepared HTTP query. The query will be properly encoded with {@link java.net.URLEncoder}
    +     * 
    +     * @throws UnsupportedEncodingException from {@link java.net.URLEncoder.encode()}
    +     */
    +    private String buildQueryFldexe(String querySql, ArrayList<String> filterConstants) throws UnsupportedEncodingException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("http://");
    +        sb.append(igniteHost);
    +        sb.append("/ignite");
    +        sb.append("?");
    +        sb.append("cmd=qryfldexe");
    +        sb.append("&");
    +        sb.append("pageSize=0");
    +        sb.append("&");
    +        if (cacheName != null) {
    +            sb.append("cacheName=");
    +            // Note that Ignite supports only "good" cache names (those that should be left unchanged by the URLEncoder.encode())
    +            sb.append(URLEncoder.encode(cacheName, "UTF-8"));
    +            sb.append("&");
    +        }
    +        int counter = 1;
    +        if (filterConstants != null) {
    --- End diff --
    
    could you please the filter construction? it looks like it creates something like
    `arg1=A&arg2=B&arg3=C`
    how do we know what field argX stands for? and if we want to filter by the last field, how can it be done? (I would think in that case we will have to pass the filterConstants with empty values as placeholders for the other fields?)
    If I understand correctly, this code is never executed because we always send filterConstants as null, so I just want to make sure it will work as expected.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171145071
  
    --- Diff: pxf/pxf-ignite/src/test/java/org/apache/hawq/pxf/plugins/ignite/IgniteFilterBuilderTest.java ---
    @@ -0,0 +1,99 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.BasicFilter;
    +import org.apache.hawq.pxf.api.FilterParser;
    +import static org.apache.hawq.pxf.api.FilterParser.Operation.*;
    +import org.apache.hawq.pxf.api.FilterParser.LogicalOperation;
    +import org.apache.hawq.pxf.api.LogicalFilter;
    +
    +import org.junit.Test;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertNull;
    +
    +public class IgniteFilterBuilderTest {
    +    @Test
    +    public void parseFilterWithThreeOperations() throws Exception {
    +        //orgin sql => col_1>'2008-02-01' and col_1<'2008-12-01' or col_2 > 1200
    +        String filterstr = "a1c25s10d2008-02-01o2a1c25s10d2008-12-01o1l0a2c20s4d1200o2l1";
    +        IgniteFilterBuilder builder = new IgniteFilterBuilder();
    +
    +        LogicalFilter filterList = (LogicalFilter) builder.getFilterObject(filterstr);
    +        assertEquals(LogicalOperation.HDOP_OR, filterList.getOperator());
    +        LogicalFilter l1_left = (LogicalFilter) filterList.getFilterList().get(0);
    +        BasicFilter l1_right = (BasicFilter) filterList.getFilterList().get(1);
    +
    +        //column_2 > 1200
    +        assertEquals(2, l1_right.getColumn().index());
    +        assertEquals(HDOP_GT, l1_right.getOperation());
    +        assertEquals(1200L, l1_right.getConstant().constant());
    +
    +        assertEquals(LogicalOperation.HDOP_AND, l1_left.getOperator());
    +        BasicFilter l2_left = (BasicFilter) l1_left.getFilterList().get(0);
    +        BasicFilter l2_right = (BasicFilter) l1_left.getFilterList().get(1);
    +
    +        //column_1 > '2008-02-01'
    +        assertEquals(1, l2_left.getColumn().index());
    +        assertEquals(HDOP_GT, l2_left.getOperation());
    +        assertEquals("2008-02-01", l2_left.getConstant().constant());
    +
    +        //column_2 < '2008-12-01'
    +        assertEquals(1, l2_right.getColumn().index());
    +        assertEquals(HDOP_LT, l2_right.getOperation());
    +        assertEquals("2008-12-01", l2_right.getConstant().constant());
    +
    +    }
    +
    +    @Test
    +    public void parseFilterWithLogicalOperation() throws Exception {
    +        WhereSQLBuilder builder = new WhereSQLBuilder(null);
    +        LogicalFilter filter = (LogicalFilter) builder.getFilterObject("a1c25s5dfirsto5a2c20s1d2o2l0");
    +        assertEquals(LogicalOperation.HDOP_AND, filter.getOperator());
    +        assertEquals(2, filter.getFilterList().size());
    +    }
    +
    +    @Test
    +    public void parseNestedExpressionWithLogicalOperation() throws Exception {
    +        WhereSQLBuilder builder = new WhereSQLBuilder(null);
    +        LogicalFilter filter = (LogicalFilter) builder.getFilterObject("a1c25s5dfirsto5a2c20s1d2o2l0a1c20s1d1o1l1");
    +        assertEquals(LogicalOperation.HDOP_OR, filter.getOperator());
    +        assertEquals(LogicalOperation.HDOP_AND, ((LogicalFilter) filter.getFilterList().get(0)).getOperator());
    +        assertEquals(HDOP_LT, ((BasicFilter) filter.getFilterList().get(1)).getOperation());
    +    }
    +
    +    @Test
    +    public void parseISNULLExpression() throws Exception {
    --- End diff --
    
    maybe change test names to `parseIsNullExpression` and `parseIsNotNullExpression`


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171129896
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,384 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +/**
    + * Fragmenter class for Ignite data resources.
    + * @note At the moment this fragmenter works just like the JDBC's one
    + *
    + * Aside from implementing PXF {@link Fragmenter},
    + * this class also transforms input data path (given by user) by adding the
    + * information on splitting the request into a few requests for different regions.
    + * <br>
    + * The parameter Patterns
    + * <br>
    + * There are three  parameters, the format is as follows:
    + * <br>
    + * <pre>
    + * <code>PARTITION_BY=column_name:column_type&amp;RANGE=start_value[:end_value]&amp;INTERVAL=interval_num[:interval_unit]</code>
    + * </pre>
    + * The <code>PARTITION_BY</code> parameter can be split by colon(':'),the <code>column_type</code> current supported : <code>date,int,enum</code> .
    + * The Date format is 'yyyy-MM-dd'. <br>
    + * The <code>RANGE</code> parameter can be split by colon(':') ,used to identify the starting range of each fragment.
    + * The range is left-closed, ie:<code> '&gt;= start_value AND &lt; end_value' </code>.If the <code>column_type</code> is <code>int</code>,
    + * the <code>end_value</code> can be empty. If the <code>column_type</code>is <code>enum</code>,the parameter <code>RANGE</code> can be empty. <br>
    + * The <code>INTERVAL</code> parameter can be split by colon(':'), indicate the interval value of one fragment.
    + * When <code>column_type</code> is <code>date</code>,this parameter must be split by colon, and <code>interval_unit</code> can be <code>year,month,day</code>.
    + * When <code>column_type</code> is <code>int</code>, the <code>interval_unit</code> can be empty.
    + * When <code>column_type</code> is <code>enum</code>,the <code>INTERVAL</code> parameter can be empty.
    + * <br>
    + * <p>
    + * The syntax examples is:
    + * <br>
    + * <code>PARTITION_BY=createdate:date&amp;RANGE=2008-01-01:2010-01-01&amp;INTERVAL=1:month'</code>
    + * <br>
    + * <code>PARTITION_BY=year:int&amp;RANGE=2008:2010&amp;INTERVAL=1</code>
    + * <br>
    + * <code>PARTITION_BY=grade:enum&amp;RANGE=excellent:good:general:bad</code>
    + * </p>
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    Log LOG = LogFactory.getLog(IgnitePartitionFragmenter.class);
    +
    +    String[] partitionBy = null;
    +    String[] range = null;
    +    String[] interval = null;
    +    PartitionType partitionType = null;
    +    String partitionColumn = null;
    +    IntervalType intervalType = null;
    +    int intervalNum = 1;
    +
    +    //when partitionType is DATE,it is valid
    +    Calendar rangeStart = null;
    +    Calendar rangeEnd = null;
    +
    +
    +    enum PartitionType {
    +        DATE,
    +        INT,
    +        ENUM;
    +
    +        public static PartitionType getType(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    enum IntervalType {
    +        DAY,
    +        MONTH,
    +        YEAR;
    +
    +        public static IntervalType type(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    /**
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inConf) throws UserDataException {
    +        super(inConf);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inConf.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inConf.getUserProperty("PARTITION_BY").split(":");
    +            partitionColumn = partitionBy[0];
    +            partitionType = PartitionType.getType(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' invalid, the pattern is 'column_name:date|int|enum'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inConf.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inConf.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.type(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    +                }
    +            } 
    +            else if (partitionType != PartitionType.ENUM) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be specified along with 'PARTITION_BY'");
    +            }
    +            if (intervalNum < 1) {
    +                throw new UserDataException("The parameter 'INTERVAL' must > 1, but actual is '" + intervalNum + "'");
    +            }
    +        } 
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'INTERVAL' invalid, the pattern is 'interval_num[:interval_unit]'");
    --- End diff --
    
    typo `is invalid`


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171115368
  
    --- Diff: pxf/pxf-service/src/configs/templates/pxf-private-hdp.classpath.template ---
    @@ -62,6 +63,8 @@ PXF_HOME/lib/pxf-jdbc.jar
     /usr/hdp/current/hadoop-client/client/snappy-java.jar
     /usr/hdp/current/hadoop-client/lib/asm-*[0-9].jar
     /usr/hdp/current/hadoop-client/lib/jersey-server-*[0-9].jar
    +/usr/hdp/current/hadoop-client/client/gson.jar
    +
    --- End diff --
    
    extra line ?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171130648
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,384 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +/**
    + * Fragmenter class for Ignite data resources.
    + * @note At the moment this fragmenter works just like the JDBC's one
    + *
    + * Aside from implementing PXF {@link Fragmenter},
    + * this class also transforms input data path (given by user) by adding the
    + * information on splitting the request into a few requests for different regions.
    + * <br>
    + * The parameter Patterns
    + * <br>
    + * There are three  parameters, the format is as follows:
    + * <br>
    + * <pre>
    + * <code>PARTITION_BY=column_name:column_type&amp;RANGE=start_value[:end_value]&amp;INTERVAL=interval_num[:interval_unit]</code>
    + * </pre>
    + * The <code>PARTITION_BY</code> parameter can be split by colon(':'),the <code>column_type</code> current supported : <code>date,int,enum</code> .
    + * The Date format is 'yyyy-MM-dd'. <br>
    + * The <code>RANGE</code> parameter can be split by colon(':') ,used to identify the starting range of each fragment.
    + * The range is left-closed, ie:<code> '&gt;= start_value AND &lt; end_value' </code>.If the <code>column_type</code> is <code>int</code>,
    + * the <code>end_value</code> can be empty. If the <code>column_type</code>is <code>enum</code>,the parameter <code>RANGE</code> can be empty. <br>
    + * The <code>INTERVAL</code> parameter can be split by colon(':'), indicate the interval value of one fragment.
    + * When <code>column_type</code> is <code>date</code>,this parameter must be split by colon, and <code>interval_unit</code> can be <code>year,month,day</code>.
    + * When <code>column_type</code> is <code>int</code>, the <code>interval_unit</code> can be empty.
    + * When <code>column_type</code> is <code>enum</code>,the <code>INTERVAL</code> parameter can be empty.
    + * <br>
    + * <p>
    + * The syntax examples is:
    + * <br>
    + * <code>PARTITION_BY=createdate:date&amp;RANGE=2008-01-01:2010-01-01&amp;INTERVAL=1:month'</code>
    + * <br>
    + * <code>PARTITION_BY=year:int&amp;RANGE=2008:2010&amp;INTERVAL=1</code>
    + * <br>
    + * <code>PARTITION_BY=grade:enum&amp;RANGE=excellent:good:general:bad</code>
    + * </p>
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    Log LOG = LogFactory.getLog(IgnitePartitionFragmenter.class);
    +
    +    String[] partitionBy = null;
    +    String[] range = null;
    +    String[] interval = null;
    +    PartitionType partitionType = null;
    +    String partitionColumn = null;
    +    IntervalType intervalType = null;
    +    int intervalNum = 1;
    +
    +    //when partitionType is DATE,it is valid
    +    Calendar rangeStart = null;
    +    Calendar rangeEnd = null;
    +
    +
    +    enum PartitionType {
    +        DATE,
    +        INT,
    +        ENUM;
    +
    +        public static PartitionType getType(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    enum IntervalType {
    +        DAY,
    +        MONTH,
    +        YEAR;
    +
    +        public static IntervalType type(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    /**
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inConf) throws UserDataException {
    +        super(inConf);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inConf.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inConf.getUserProperty("PARTITION_BY").split(":");
    +            partitionColumn = partitionBy[0];
    +            partitionType = PartitionType.getType(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' invalid, the pattern is 'column_name:date|int|enum'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inConf.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inConf.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.type(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    +                }
    +            } 
    +            else if (partitionType != PartitionType.ENUM) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be specified along with 'PARTITION_BY'");
    +            }
    +            if (intervalNum < 1) {
    +                throw new UserDataException("The parameter 'INTERVAL' must > 1, but actual is '" + intervalNum + "'");
    +            }
    +        } 
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'INTERVAL' invalid, the pattern is 'interval_num[:interval_unit]'");
    +        }
    +
    +        //parse any date values
    +        try {
    +            if (partitionType == PartitionType.DATE) {
    +                SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd");
    +                rangeStart = Calendar.getInstance();
    +                rangeStart.setTime(df.parse(range[0]));
    +                rangeEnd = Calendar.getInstance();
    +                rangeEnd.setTime(df.parse(range[1]));
    +            }
    +        } catch (ParseException e) {
    +            throw new UserDataException("The parameter 'RANGE' has invalid date format. Expected format is 'YYYY-MM-DD'");
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful; some partition used");
    +        }
    +    }
    +
    +    /**
    +     * Returns statistics for  Ignite table. Currently it's not implemented.
    +     * @throws UnsupportedOperationException
    +     */
    +    @Override
    +    public FragmentsStats getFragmentsStats() throws UnsupportedOperationException {
    +        throw new UnsupportedOperationException("ANALYZE for Ignite plugin is not supported");
    +    }
    +
    +    /**
    +     * Returns list of fragments for Ignite table queries
    +     *
    +     * @return a list of fragments
    +     * @throws UnknownHostException from java.net.InetAddress.getLocalHost().getHostAddress()
    +     */
    +    @Override
    +    public List<Fragment> getFragments() throws UnknownHostException {
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() called; dataSource is '" + inputData.getDataSource() + "'");
    +        }
    +
    +        // Always use 'localhost' as a replica holder
    +        // The replica holder is a *PXF* host which can process the requested query
    +        // Note that this is not an Ignite host
    +        String[] replicaHostAddressWrapped = new String[]{InetAddress.getLocalHost().getHostAddress()};
    +        byte[] fragmentMetadata = null;
    +        byte[] fragmentUserdata = null;
    +
    +        if (partitionType == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() found no partition");
    +            }
    +            Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +            fragments.add(fragment);
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() successful. Total fragments: " + fragments.size());
    +            }
    +            return fragments;
    +        }
    +
    +        switch (partitionType) {
    +            case DATE: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found DATE partition");
    +                }
    +                int currInterval = intervalNum;
    +
    +                Calendar fragStart = rangeStart;
    +                while (fragStart.before(rangeEnd)) {
    +                    Calendar fragEnd = (Calendar) fragStart.clone();
    +                    switch (intervalType) {
    +                        case DAY:
    +                            fragEnd.add(Calendar.DAY_OF_MONTH, currInterval);
    +                            break;
    +                        case MONTH:
    +                            fragEnd.add(Calendar.MONTH, currInterval);
    +                            break;
    +                        case YEAR:
    +                            fragEnd.add(Calendar.YEAR, currInterval);
    +                            break;
    +                    }
    +                    if (fragEnd.after(rangeEnd))
    +                        fragEnd = (Calendar) rangeEnd.clone();
    +
    +                    // Build Fragment.metadata: convert the date to a millisecond, then get bytes.
    +                    byte[] msStart = ByteUtil.getBytes(fragStart.getTimeInMillis());
    +                    byte[] msEnd = ByteUtil.getBytes(fragEnd.getTimeInMillis());
    +                    fragmentMetadata = ArrayUtils.addAll(msStart, msEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;
    +                }
    +                break;
    +            }
    +            case INT: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found INT partition");
    +                }
    +                int rangeStart = Integer.parseInt(range[0]);
    +                int rangeEnd = Integer.parseInt(range[1]);
    +                int currInterval = intervalNum;
    +
    +                // Validate : curr_interval > 0
    +                int fragStart = rangeStart;
    +                while (fragStart < rangeEnd) {
    +                    int fragEnd = fragStart + currInterval;
    +                    if (fragEnd > rangeEnd) fragEnd = rangeEnd;
    --- End diff --
    
    please break to multiple lines:
    ```
    if (fragEnd > rangeEnd) {
       fragEnd = rangeEnd;
    }
    ```


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171680865
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,411 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +
    +/**
    + * PXF-Ignite fragmenter class
    + * 
    + * This fragmenter works just like the one in PXF JDBC plugin
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    /**
    +     * Class constructor
    +     * 
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inputData) throws UserDataException {
    +        super(inputData);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inputData.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inputData.getUserProperty("PARTITION_BY").split(":");
    +            partitionType = PartitionType.typeOf(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' is invalid. The pattern is 'column_name:DATE|INT|ENUM'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inputData.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' is invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inputData.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.typeOf(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    +                }
    +            } 
    +            else if (partitionType != PartitionType.ENUM) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be specified along with 'PARTITION_BY'");
    +            }
    +            if (intervalNum < 1) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be at least 1. The actual is '" + intervalNum + "'");
    +            }
    +        } 
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'INTERVAL' invalid. The pattern is 'interval_num[:interval_unit]'");
    +        }
    +
    +        //parse date values
    +        try {
    +            if (partitionType == PartitionType.DATE) {
    +                SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd");
    +                rangeStart = Calendar.getInstance();
    +                rangeStart.setTime(df.parse(range[0]));
    +                rangeEnd = Calendar.getInstance();
    +                rangeEnd.setTime(df.parse(range[1]));
    +            }
    +        } catch (ParseException e) {
    +            throw new UserDataException("The parameter 'RANGE' has invalid date format. Expected format is 'yyyy-MM-dd'");
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful; some partition used");
    +        }
    +    }
    +
    +    /**
    +     * Returns statistics for the Ignite table. This is not implemented in the current version
    +     * @throws UnsupportedOperationException
    +     */
    +    @Override
    +    public FragmentsStats getFragmentsStats() throws UnsupportedOperationException {
    +        throw new UnsupportedOperationException("ANALYZE for Ignite plugin is not supported");
    +    }
    +
    +    /**
    +     * Returns list of fragments for Ignite table queries
    +     *
    +     * @throws UnsupportedOperationException if a partition of unknown type was found
    +     * 
    +     * @return a list of fragments
    +     */
    +    @Override
    +    public List<Fragment> getFragments() throws UnsupportedOperationException {
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() called; dataSource is '" + inputData.getDataSource() + "'");
    +        }
    +
    +        byte[] fragmentMetadata = null;
    +        byte[] fragmentUserdata = null;
    +
    +        if (partitionType == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() found no partition");
    +            }
    +            Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +            fragments.add(fragment);
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() successful");
    +            }
    +            return fragments;
    +        }
    +
    +        switch (partitionType) {
    +            case DATE: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found DATE partition");
    +                }
    +                int currInterval = intervalNum;
    +
    +                Calendar fragStart = rangeStart;
    +                while (fragStart.before(rangeEnd)) {
    +                    Calendar fragEnd = (Calendar) fragStart.clone();
    +                    switch (intervalType) {
    +                        case DAY:
    +                            fragEnd.add(Calendar.DAY_OF_MONTH, currInterval);
    +                            break;
    +                        case MONTH:
    +                            fragEnd.add(Calendar.MONTH, currInterval);
    +                            break;
    +                        case YEAR:
    +                            fragEnd.add(Calendar.YEAR, currInterval);
    +                            break;
    +                    }
    +                    if (fragEnd.after(rangeEnd))
    +                        fragEnd = (Calendar) rangeEnd.clone();
    +
    +                    // Note that the date is stored in milliseconds
    +                    byte[] msStart = ByteUtil.getBytes(fragStart.getTimeInMillis());
    +                    byte[] msEnd = ByteUtil.getBytes(fragEnd.getTimeInMillis());
    +                    fragmentMetadata = ArrayUtils.addAll(msStart, msEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;
    +                }
    +                break;
    +            }
    +            case INT: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found INT partition");
    +                }
    +                int rangeStart = Integer.parseInt(range[0]);
    +                int rangeEnd = Integer.parseInt(range[1]);
    +                int currInterval = intervalNum;
    +
    +                int fragStart = rangeStart;
    +                while (fragStart < rangeEnd) {
    +                    int fragEnd = fragStart + currInterval;
    +                    if (fragEnd > rangeEnd) {
    +                        fragEnd = rangeEnd;
    +                    }
    +
    +                    byte[] bStart = ByteUtil.getBytes(fragStart);
    +                    byte[] bEnd = ByteUtil.getBytes(fragEnd);
    +                    fragmentMetadata = ArrayUtils.addAll(bStart, bEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;
    +                }
    +                break;
    +            }
    +            case ENUM: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found ENUM partition");
    +                }
    +                for (String frag : range) {
    +                    fragmentMetadata = frag.getBytes();
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +                }
    +                break;
    +            }
    +            default: {
    +                throw new UnsupportedOperationException("getFragments() found a partition of unknown type and failed");
    +            }
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() successful");
    +        }
    +        return fragments;
    +    }
    +
    +    /**
    +     * Insert partition constraints into the prepared SQL query.
    +     * 
    +     * @param inputData pre-validated PXF InputData
    +     * @param sb the SQL query that is prepared for appending WHERE constraints.
    +     * Other SQL statements may be present, but they must be complete. Note that no check is performed to check their "completeness"
    +     */
    +    public static StringBuilder buildFragmenterSql(InputData inputData, StringBuilder sb) {
    --- End diff --
    
    Since you are mutating initial query - I think it would be cleaner to do it in place and return `void`.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171074788
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePlugin.java ---
    @@ -0,0 +1,90 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.utilities.Plugin;
    +
    +/**
    + * This class resolves the jdbc connection parameter and manages the opening and closing of the jdbc connection.
    + * Implemented subclasses: {@link IgniteReadAccessor}.
    --- End diff --
    
    typo IgniteReadAccessor -> IgniteAccessor.
    (I thought the compilation checks for documentation errors)


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171117000
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteResolver.java ---
    @@ -0,0 +1,173 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +
    +import java.util.ArrayList;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.text.SimpleDateFormat;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.codec.binary.Base64;
    +import org.apache.commons.codec.binary.Hex;
    +
    +import com.google.gson.JsonArray;
    +
    +
    +
    +/**
    + * PXF-Ignite resolver class
    + */
    +public class IgniteResolver extends IgnitePlugin implements ReadResolver, WriteResolver {
    +    private static final Log LOG = LogFactory.getLog(IgniteResolver.class);
    +    
    +    // HAWQ column descriptors
    +    private ArrayList<ColumnDescriptor> columns = null;
    +
    +    public IgniteResolver(InputData input) throws Exception {
    +        super(input);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        columns = input.getTupleDescription();
    +        
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful");
    +        }
    +    }
    +
    +    /**
    +     * Transform a JsonArray object stored in {@link OneRow} into a List {@link OneField}
    --- End diff --
    
    typo `a list of OneField`


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171053913
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    +        return true;
    +    }
    +
    +    /**
    +     * readNextObject() implementation
    +     */
    +    @Override
    +    public OneRow readNextObject() throws Exception {
    +        if (urlReadFetch == null) {
    +            LOG.error("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +            throw new ProtocolException("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +        }
    +
    +        if (bufferRead.isEmpty()) {
    +            // Refill buffer
    +            if (isLastReadFinished) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +
    +            JsonElement response = sendRestRequest(urlReadFetch);
    +            isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +
    +            // Parse 'items'
    +            Iterator<JsonElement> itemsIterator = response.getAsJsonObject().get("items").getAsJsonArray().iterator();
    +            while (itemsIterator.hasNext()) {
    +                if (!bufferRead.add(itemsIterator.next().getAsJsonArray())) {
    +                    LOG.error("readNextObject(): Buffer refill failed (not enough memory in 'bufferRead')");
    +                    throw new OutOfMemoryError("readNextObject(): not enough memory in 'bufferRead'");
    +                }
    +            }
    +
    +            // Check again in case "response" contains no elements
    +            if (bufferRead.isEmpty()) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): Buffer refill failed");
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +        }
    +
    +        return new OneRow(bufferRead.pollFirst());
    +    }
    +
    +    /**
    +     * closeForRead() implementation
    +     */
    +    @Override
    +    public void closeForRead() {
    +        if (urlReadClose != null) {
    +            try {
    +                sendRestRequest(urlReadClose);
    +            }
    +            catch (Exception e) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("closeForRead() Exception: " + e.getClass().getSimpleName());
    +                }
    +            }
    +        }
    +        isLastReadFinished = false;
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Ignite read request finished. URL: '" + urlReadClose + "'");
    +        }
    +    }
    +
    +    /**
    +     * openForWrite() implementation.
    +     * No queries are sent to Ignite by this procedure, so if there are some problems (for example, with connection), they will be revealed only during the execution of 'writeNextObject()'
    +     */
    +    @Override
    +    public boolean openForWrite() throws UserDataException {
    +        // This is a temporary solution. At the moment there is no other way (except for the usage of user-defined parameters) to get the correct name of Ignite table: GPDB inserts extra data into the address, as required by Hadoop.
    +        // Note that if no extra data is present, the 'definedSource' will be left unchanged
    +        String definedSource = inputData.getDataSource();
    +        Pattern pattern = Pattern.compile("/(.*)/[0-9]*-[0-9]*_[0-9]*");
    +        Matcher matcher = pattern.matcher(definedSource);
    +        if (matcher.find()) {
    +            inputData.setDataSource(matcher.group(1));
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("INSERT INTO ");
    +
    +        // Insert the table name
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert the column names
    +        sb.append("(");
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        String fieldDivisor = "";
    +        for (int i = 0; i < columns.size(); i++) {
    +            sb.append(fieldDivisor);
    +            fieldDivisor = ", ";
    +            sb.append(columns.get(i).columnName());
    +        }
    +        sb.append(")");
    +
    +        sb.append(" VALUES ");
    +
    +        queryWrite = sb.toString();
    +        return true;
    +    }
    +
    +    /**
    +     * writeNextObject() implementation
    +     */
    +    @Override
    +    public boolean writeNextObject(OneRow currentRow) throws Exception {
    +        boolean currentRowInBuffer = bufferWrite.add(currentRow);
    +
    +        if (!isWriteActive) {
    +            if (!currentRowInBuffer) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferWrite')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferWrite'");
    +            }
    +            LOG.info("Ignite write request. Query: '" + queryWrite + "'");
    +            sendInsertRestRequest(queryWrite);
    +            bufferWrite.removeFirst();
    +            isWriteActive = true;
    +            return true;
    +        }
    +
    +        if ((bufferWrite.size() >= bufferSize) || (!currentRowInBuffer)) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +
    +        if (!currentRowInBuffer) {
    +            if (!bufferWrite.add(currentRow)) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferSend')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferSend'");
    +            }
    +        }
    +        
    +        return true;
    +    }
    +
    +    /**
    +     * closeForWrite() implementation
    +     */
    +    @Override
    +    public void closeForWrite() throws Exception {
    +        if (!bufferWrite.isEmpty()) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +        if (isWriteActive) {
    +            // At this point, the request must have finished successfully; otherwise an exception would be thrown before
    +            LOG.info("Ignite write request finished successfully. Query: '" + queryWrite + "'");
    +        }
    +        isWriteActive = false;
    +    }
    +
    +    /**
    +     * Build HTTP GET query for Ignite REST API with command 'qryfldexe'
    +     * 
    +     * @param querySql SQL query with filter constants. The constants must replaced by "?" if 'filterConstants' is not given
    +     * @param filterConstants
    +     * 
    +     * @return Prepared HTTP query. The query will be properly encoded with {@link java.net.URLEncoder}
    +     * 
    +     * @throws UnsupportedEncodingException from {@link java.net.URLEncoder.encode()}
    +     */
    +    private String buildQueryFldexe(String querySql, ArrayList<String> filterConstants) throws UnsupportedEncodingException {
    --- End diff --
    
    `filterConstants` parameter doesn't have to be `ArrayList`, just`List` should be enough)


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171114496
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePlugin.java ---
    @@ -0,0 +1,90 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.utilities.Plugin;
    +
    +/**
    + * This class resolves the jdbc connection parameter and manages the opening and closing of the jdbc connection.
    + * Implemented subclasses: {@link IgniteReadAccessor}.
    + */
    +public class IgnitePlugin extends Plugin {
    +    private static final Log LOG = LogFactory.getLog(IgnitePlugin.class);
    +
    +    // Ignite cache
    +    protected static final String igniteHostDefault = "127.0.0.1:8080";
    +    protected String igniteHost = null;
    +    // PXF buffer for Ignite data. '0' is allowed for INSERT queries
    +    protected static final int bufferSizeDefault = 128;
    +    protected int bufferSize = bufferSizeDefault;
    +    // Ignite cache name
    +    protected String cacheName = null;
    +
    +    /**
    +     * Parse and check the InputData
    +     * @param inputData
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePlugin(InputData inputData) throws UserDataException {
    +        super(inputData);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +        
    +        igniteHost = inputData.getUserProperty("IGNITE_HOST");
    +        if (igniteHost == null) {
    +            igniteHost = igniteHostDefault;
    +        }
    +
    +        cacheName = inputData.getUserProperty("IGNITE_CACHE");
    +        if (cacheName == null) {
    --- End diff --
    
    remove this ?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r170725980
  
    --- Diff: pxf/pxf-ignite/README.md ---
    @@ -0,0 +1,65 @@
    +# Accessing Ignite database using PXF
    +
    +The PXF Ignite plug-in enables to access the [Ignite database](https://ignite.apache.org/) (both read and write operations are supported) via REST API.
    +
    +
    +# Prerequisites
    +
    +Check the following before using the plug-in:
    +
    +* The Ignite plug-in is installed on all PXF nodes;
    +
    +* The Ignite client is installed and running at the `IGNITE_HOST` (`localhost` by default; see below), and it accepts http queries from the PXF (note that *enabling Ignite REST API does not require changes in Ignite configuration*; see the instruction on how to do that at https://apacheignite.readme.io/docs/rest-api#section-getting-started).
    +
    +
    +# Syntax
    +
    +```
    +CREATE [READABLE] EXTERNAL TABLE <table_name> (
    +    <column_name> <data_type>[, <column_name> <data_type>, ...] | LIKE <other_table>
    +)
    +LOCATION ('pxf://<ignite_table_name>?PROFILE=Ignite[&<extra-parameter>&<extra-parameter>&...]')
    +FORMAT 'CUSTOM' (formatter='pxfwritable_import');
    +```
    +where each `<extra-parameter>` is one of the following:
    +* `IGNITE_HOST=<ignite_host_address_with_port>`. The location of Ignite client node. If not given, `127.0.0.1:8080` is used by default;
    +* `IGNITE_CACHE=<ignite_cache_name>`. The name of Ignite cache to use. If not given, this parameter is not included in queries from PXF to Ignite, thus Ignite default values will be used (at the moment, this is `Default` cache). This option is **case-sensitive**;
    +* `BUFFER_SIZE=<unsigned_int>`. The number of tuples send to (from) Ignite per a response. The same number of tuples is stored in in-plug-in cache. The values `0` and `1` are equal (cache is not used, each tuple is passed in it's own query to Ignite). If not given, `128` is used by default;
    +* `PARTITION_BY=<column>:<column_type>`. See below;
    +* `RANGE=<start_value>:<end_value>`. See below;
    +* `INTERVAL=<value>[:<unit>]`. See below.
    +
    +
    +# Partitioning
    +## Introduction
    +
    +PXF Ignite plugin supports simultaneous access to Ignite database from multiple PXF segments. *Partitioning* should be used in order to perform such operation.
    +
    +Partitioning in PXF Ignite plug-in works just like in PXF JDBC plug-in.
    +
    +This feature is optional. However, a bug in the `pxf-service` which makes partitioning necessary for any query was fixed only on 17th Jan 2018 in [this commit](https://github.com/apache/incubator-hawq/commit/0d620e431026834dd70c9e0d63edf8bb28b38227), so the older versions of PXF may return an exception if a query does not contain a meaningful `PARTITION_BY` parameter.
    --- End diff --
    
    All the data is retrieved by a single PXF segment.
    
    Added this to README.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171077045
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    --- End diff --
    
    maybe add a space after the comma for clarity `", "`


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171039740
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/ByteUtil.java ---
    @@ -0,0 +1,77 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +/**
    + * A tool class, used to deal with byte array split and conversions.
    + * IgnitePartitionFragmenter requires this class in order to work with 'fragmentMetadata' field
    + */
    +public class ByteUtil {
    +    public static byte[][] splitBytes(byte[] bytes, int n) {
    +        int len = bytes.length / n;
    +        byte[][] newBytes = new byte[len][];
    +        int j = 0;
    +        for (int i = 0; i < len; i++) {
    +            newBytes[i] = new byte[n];
    +            for (int k = 0; k < n; k++) newBytes[i][k] = bytes[j++];
    +        }
    +        return newBytes;
    +    }
    +
    +    public static byte[] getBytes(long value) {
    --- End diff --
    
    why not use ByteUtils from ApacheCommons ?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171521898
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/ByteUtil.java ---
    @@ -0,0 +1,77 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +/**
    + * A tool class, used to deal with byte array split and conversions.
    + * IgnitePartitionFragmenter requires this class in order to work with 'fragmentMetadata' field
    + */
    +public class ByteUtil {
    +    public static byte[][] splitBytes(byte[] bytes, int n) {
    +        int len = bytes.length / n;
    +        byte[][] newBytes = new byte[len][];
    +        int j = 0;
    +        for (int i = 0; i < len; i++) {
    +            newBytes[i] = new byte[n];
    +            for (int k = 0; k < n; k++) newBytes[i][k] = bytes[j++];
    +        }
    +        return newBytes;
    +    }
    +
    +    public static byte[] getBytes(long value) {
    --- End diff --
    
    Done. Thank you for the comment.
    
    This also eliminates the dependency on Apache Commons ArrayUtils.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171042814
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    --- End diff --
    
    this should probably be debug level, otherwise each query will leave a log trace ?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171053664
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    +        return true;
    +    }
    +
    +    /**
    +     * readNextObject() implementation
    +     */
    +    @Override
    +    public OneRow readNextObject() throws Exception {
    +        if (urlReadFetch == null) {
    +            LOG.error("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +            throw new ProtocolException("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +        }
    +
    +        if (bufferRead.isEmpty()) {
    +            // Refill buffer
    +            if (isLastReadFinished) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +
    +            JsonElement response = sendRestRequest(urlReadFetch);
    +            isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +
    +            // Parse 'items'
    +            Iterator<JsonElement> itemsIterator = response.getAsJsonObject().get("items").getAsJsonArray().iterator();
    +            while (itemsIterator.hasNext()) {
    +                if (!bufferRead.add(itemsIterator.next().getAsJsonArray())) {
    +                    LOG.error("readNextObject(): Buffer refill failed (not enough memory in 'bufferRead')");
    +                    throw new OutOfMemoryError("readNextObject(): not enough memory in 'bufferRead'");
    +                }
    +            }
    +
    +            // Check again in case "response" contains no elements
    +            if (bufferRead.isEmpty()) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): Buffer refill failed");
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +        }
    +
    +        return new OneRow(bufferRead.pollFirst());
    +    }
    +
    +    /**
    +     * closeForRead() implementation
    +     */
    +    @Override
    +    public void closeForRead() {
    +        if (urlReadClose != null) {
    +            try {
    +                sendRestRequest(urlReadClose);
    +            }
    +            catch (Exception e) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("closeForRead() Exception: " + e.getClass().getSimpleName());
    +                }
    +            }
    +        }
    +        isLastReadFinished = false;
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Ignite read request finished. URL: '" + urlReadClose + "'");
    +        }
    +    }
    +
    +    /**
    +     * openForWrite() implementation.
    +     * No queries are sent to Ignite by this procedure, so if there are some problems (for example, with connection), they will be revealed only during the execution of 'writeNextObject()'
    +     */
    +    @Override
    +    public boolean openForWrite() throws UserDataException {
    +        // This is a temporary solution. At the moment there is no other way (except for the usage of user-defined parameters) to get the correct name of Ignite table: GPDB inserts extra data into the address, as required by Hadoop.
    +        // Note that if no extra data is present, the 'definedSource' will be left unchanged
    +        String definedSource = inputData.getDataSource();
    +        Pattern pattern = Pattern.compile("/(.*)/[0-9]*-[0-9]*_[0-9]*");
    +        Matcher matcher = pattern.matcher(definedSource);
    +        if (matcher.find()) {
    +            inputData.setDataSource(matcher.group(1));
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("INSERT INTO ");
    +
    +        // Insert the table name
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert the column names
    +        sb.append("(");
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        String fieldDivisor = "";
    +        for (int i = 0; i < columns.size(); i++) {
    +            sb.append(fieldDivisor);
    +            fieldDivisor = ", ";
    +            sb.append(columns.get(i).columnName());
    +        }
    +        sb.append(")");
    +
    +        sb.append(" VALUES ");
    +
    +        queryWrite = sb.toString();
    +        return true;
    +    }
    +
    +    /**
    +     * writeNextObject() implementation
    +     */
    +    @Override
    +    public boolean writeNextObject(OneRow currentRow) throws Exception {
    +        boolean currentRowInBuffer = bufferWrite.add(currentRow);
    +
    +        if (!isWriteActive) {
    +            if (!currentRowInBuffer) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferWrite')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferWrite'");
    +            }
    +            LOG.info("Ignite write request. Query: '" + queryWrite + "'");
    +            sendInsertRestRequest(queryWrite);
    +            bufferWrite.removeFirst();
    +            isWriteActive = true;
    +            return true;
    +        }
    +
    +        if ((bufferWrite.size() >= bufferSize) || (!currentRowInBuffer)) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +
    +        if (!currentRowInBuffer) {
    +            if (!bufferWrite.add(currentRow)) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferSend')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferSend'");
    +            }
    +        }
    +        
    +        return true;
    +    }
    +
    +    /**
    +     * closeForWrite() implementation
    +     */
    +    @Override
    +    public void closeForWrite() throws Exception {
    +        if (!bufferWrite.isEmpty()) {
    +            sendInsertRestRequest(queryWrite);
    --- End diff --
    
    can this fail ? If yes, the exception will be thrown and isWriteActive will remain as true ?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171116876
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePlugin.java ---
    @@ -0,0 +1,90 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.utilities.Plugin;
    +
    +/**
    + * This class resolves the jdbc connection parameter and manages the opening and closing of the jdbc connection.
    + * Implemented subclasses: {@link IgniteReadAccessor}.
    --- End diff --
    
    Also please add IgniteResolver to the list.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171324542
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteResolver.java ---
    @@ -0,0 +1,173 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +
    +import java.util.ArrayList;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.text.SimpleDateFormat;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.codec.binary.Base64;
    +import org.apache.commons.codec.binary.Hex;
    +
    +import com.google.gson.JsonArray;
    +
    +
    +
    +/**
    + * PXF-Ignite resolver class
    + */
    +public class IgniteResolver extends IgnitePlugin implements ReadResolver, WriteResolver {
    +    private static final Log LOG = LogFactory.getLog(IgniteResolver.class);
    +    
    +    // HAWQ column descriptors
    +    private ArrayList<ColumnDescriptor> columns = null;
    +
    +    public IgniteResolver(InputData input) throws Exception {
    +        super(input);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        columns = input.getTupleDescription();
    +        
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful");
    +        }
    +    }
    +
    +    /**
    +     * Transform a JsonArray object stored in {@link OneRow} into a List {@link OneField}
    +     */
    +    @Override
    +    public List<OneField> getFields(OneRow row) throws Exception {
    +        JsonArray result = (JsonArray)row.getData();
    +        LinkedList<OneField> fields = new LinkedList<OneField>();
    +
    +        for (int i = 0; i < columns.size(); i++) {
    +            Object value = null;
    +            OneField oneField = new OneField(columns.get(i).columnTypeCode(), null);
    +
    +            // Handle null values
    +            if (result.get(i).isJsonNull()) {
    +                oneField.val = null;
    +                fields.add(oneField);
    +                continue;
    +            }
    +            switch (DataType.get(oneField.type)) {
    +                case INTEGER:
    +                    value = result.get(i).getAsInt();
    +                    break;
    +                case FLOAT8:
    +                    value = result.get(i).getAsDouble();
    +                    break;
    +                case REAL:
    +                    value = result.get(i).getAsFloat();
    +                    break;
    +                case BIGINT:
    +                    value = result.get(i).getAsLong();
    +                    break;
    +                case SMALLINT:
    +                    value = result.get(i).getAsShort();
    +                    break;
    +                case BOOLEAN:
    +                    value = result.get(i).getAsBoolean();
    +                    break;
    +                case VARCHAR:
    +                case BPCHAR:
    +                case TEXT:
    +                case NUMERIC:
    +                    value = result.get(i).getAsString();
    +                    break;
    +                case BYTEA:
    +                    value = Base64.decodeBase64(result.get(i).getAsString());
    +                    break;
    +                case TIMESTAMP:
    +                    value = new SimpleDateFormat("MMM d, yyyy hh:mm:ss a").parse(result.get(i).getAsString());
    +                    break;
    +                case DATE:
    +                    value = new SimpleDateFormat("yyyy-MM-dd").parse(result.get(i).getAsString());
    +                    break;
    +                default:
    +                    throw new UnsupportedOperationException("Field type not supported: " + DataType.get(oneField.type).toString()
    +                            + ", Column : " + columns.get(i).columnName());
    +            }
    +
    +            oneField.val = value;
    +            fields.add(oneField);
    +        }
    +
    +        return fields;
    +    }
    +
    +    /**
    +     * Create a query string from a List<OneField> for Ignite database REST API containing INSERT query
    +     */
    +    @Override
    +    public OneRow setFields(List<OneField> record) throws Exception {
    +        StringBuilder sb = new StringBuilder();
    +        String fieldDivisor = "";
    +        
    +        sb.append("(");
    +        for (OneField oneField : record) {
    +            sb.append(fieldDivisor);
    +            fieldDivisor = ", ";
    +            switch (DataType.get(oneField.type)) {
    +                case INTEGER:
    +                case FLOAT8:
    +                case REAL:
    +                case BIGINT:
    +                case SMALLINT:
    +                case BOOLEAN:
    +                    sb.append(String.valueOf(oneField.val));
    +                    break;
    +                case VARCHAR:
    +                case BPCHAR:
    +                case TEXT:
    +                case NUMERIC:
    --- End diff --
    
    Fixed: https://github.com/arenadata/incubator-hawq/blob/a23df5964ecee25fefd8509da0169026c2f7ac0d/pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteResolver.java#L154


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171054930
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    +        return true;
    +    }
    +
    +    /**
    +     * readNextObject() implementation
    +     */
    +    @Override
    +    public OneRow readNextObject() throws Exception {
    +        if (urlReadFetch == null) {
    +            LOG.error("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +            throw new ProtocolException("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +        }
    +
    +        if (bufferRead.isEmpty()) {
    +            // Refill buffer
    +            if (isLastReadFinished) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +
    +            JsonElement response = sendRestRequest(urlReadFetch);
    +            isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +
    +            // Parse 'items'
    +            Iterator<JsonElement> itemsIterator = response.getAsJsonObject().get("items").getAsJsonArray().iterator();
    +            while (itemsIterator.hasNext()) {
    +                if (!bufferRead.add(itemsIterator.next().getAsJsonArray())) {
    +                    LOG.error("readNextObject(): Buffer refill failed (not enough memory in 'bufferRead')");
    +                    throw new OutOfMemoryError("readNextObject(): not enough memory in 'bufferRead'");
    +                }
    +            }
    +
    +            // Check again in case "response" contains no elements
    +            if (bufferRead.isEmpty()) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): Buffer refill failed");
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +        }
    +
    +        return new OneRow(bufferRead.pollFirst());
    +    }
    +
    +    /**
    +     * closeForRead() implementation
    +     */
    +    @Override
    +    public void closeForRead() {
    +        if (urlReadClose != null) {
    +            try {
    +                sendRestRequest(urlReadClose);
    +            }
    +            catch (Exception e) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("closeForRead() Exception: " + e.getClass().getSimpleName());
    +                }
    +            }
    +        }
    +        isLastReadFinished = false;
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Ignite read request finished. URL: '" + urlReadClose + "'");
    +        }
    +    }
    +
    +    /**
    +     * openForWrite() implementation.
    +     * No queries are sent to Ignite by this procedure, so if there are some problems (for example, with connection), they will be revealed only during the execution of 'writeNextObject()'
    +     */
    +    @Override
    +    public boolean openForWrite() throws UserDataException {
    +        // This is a temporary solution. At the moment there is no other way (except for the usage of user-defined parameters) to get the correct name of Ignite table: GPDB inserts extra data into the address, as required by Hadoop.
    +        // Note that if no extra data is present, the 'definedSource' will be left unchanged
    +        String definedSource = inputData.getDataSource();
    +        Pattern pattern = Pattern.compile("/(.*)/[0-9]*-[0-9]*_[0-9]*");
    +        Matcher matcher = pattern.matcher(definedSource);
    +        if (matcher.find()) {
    +            inputData.setDataSource(matcher.group(1));
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("INSERT INTO ");
    +
    +        // Insert the table name
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert the column names
    +        sb.append("(");
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        String fieldDivisor = "";
    +        for (int i = 0; i < columns.size(); i++) {
    +            sb.append(fieldDivisor);
    +            fieldDivisor = ", ";
    +            sb.append(columns.get(i).columnName());
    +        }
    +        sb.append(")");
    +
    +        sb.append(" VALUES ");
    +
    +        queryWrite = sb.toString();
    +        return true;
    +    }
    +
    +    /**
    +     * writeNextObject() implementation
    +     */
    +    @Override
    +    public boolean writeNextObject(OneRow currentRow) throws Exception {
    +        boolean currentRowInBuffer = bufferWrite.add(currentRow);
    +
    +        if (!isWriteActive) {
    +            if (!currentRowInBuffer) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferWrite')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferWrite'");
    +            }
    +            LOG.info("Ignite write request. Query: '" + queryWrite + "'");
    +            sendInsertRestRequest(queryWrite);
    +            bufferWrite.removeFirst();
    +            isWriteActive = true;
    +            return true;
    +        }
    +
    +        if ((bufferWrite.size() >= bufferSize) || (!currentRowInBuffer)) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +
    +        if (!currentRowInBuffer) {
    +            if (!bufferWrite.add(currentRow)) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferSend')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferSend'");
    +            }
    +        }
    +        
    +        return true;
    +    }
    +
    +    /**
    +     * closeForWrite() implementation
    +     */
    +    @Override
    +    public void closeForWrite() throws Exception {
    +        if (!bufferWrite.isEmpty()) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +        if (isWriteActive) {
    +            // At this point, the request must have finished successfully; otherwise an exception would be thrown before
    +            LOG.info("Ignite write request finished successfully. Query: '" + queryWrite + "'");
    +        }
    +        isWriteActive = false;
    +    }
    +
    +    /**
    +     * Build HTTP GET query for Ignite REST API with command 'qryfldexe'
    +     * 
    +     * @param querySql SQL query with filter constants. The constants must replaced by "?" if 'filterConstants' is not given
    +     * @param filterConstants
    +     * 
    +     * @return Prepared HTTP query. The query will be properly encoded with {@link java.net.URLEncoder}
    +     * 
    +     * @throws UnsupportedEncodingException from {@link java.net.URLEncoder.encode()}
    +     */
    +    private String buildQueryFldexe(String querySql, ArrayList<String> filterConstants) throws UnsupportedEncodingException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("http://");
    +        sb.append(igniteHost);
    +        sb.append("/ignite");
    +        sb.append("?");
    +        sb.append("cmd=qryfldexe");
    +        sb.append("&");
    +        sb.append("pageSize=0");
    +        sb.append("&");
    +        if (cacheName != null) {
    +            sb.append("cacheName=");
    +            // Note that Ignite supports only "good" cache names (those that should be left unchanged by the URLEncoder.encode())
    +            sb.append(URLEncoder.encode(cacheName, "UTF-8"));
    +            sb.append("&");
    +        }
    +        int counter = 1;
    +        if (filterConstants != null) {
    +            for (String constant : filterConstants) {
    +                sb.append("arg");
    +                sb.append(counter);
    +                sb.append("=");
    +                sb.append(URLEncoder.encode(constant, "UTF-8"));
    +                sb.append("&");
    +                counter += 1;
    +            }
    +        }
    +        sb.append("qry=");
    +        sb.append(URLEncoder.encode(querySql, "UTF-8"));
    +
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Build HTTP GET query for Ignite REST API with command 'qryfetch'
    +     * This query is used to retrieve data after the 'qryfldexe' command started
    +     * 
    +     * @param queryId ID of the query assigned by Ignite when the query started
    +     *
    +     * @return Prepared HTTP query
    +     */
    +    private String buildQueryFetch(int queryId) {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("http://");
    +        sb.append(igniteHost);
    +        sb.append("/ignite");
    +        sb.append("?");
    +        sb.append("cmd=qryfetch");
    +        sb.append("&");
    +        sb.append("pageSize=");
    +        sb.append(bufferSize);
    +        sb.append("&");
    +        sb.append("qryId=");
    +        sb.append(queryId);
    +
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Build HTTP GET query for Ignite REST API with command 'qrycls'
    +     * This query is used to close query resources on Ignite side
    +     * 
    +     * @param queryId ID of the query assigned by Ignite when the query started
    +     *
    +     * @return Prepared HTTP query
    +     */
    +    private String buildQueryCls(int queryId) {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("http://");
    +        sb.append(igniteHost);
    +        sb.append("/ignite");
    +        sb.append("?");
    +        sb.append("cmd=qrycls");
    +        sb.append("&");
    +        sb.append("qryId=");
    +        sb.append(queryId);
    +
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Send a REST request to the Ignite server
    +     * 
    +     * @param query A prepared and properly encoded HTTP GET request
    +     * 
    +     * @return "response" field from the received JSON object
    +     * (See Ignite REST API documentation for details)
    +     * 
    +     * @throws ProtocolException if Ignite reports error in it's JSON response
    +     * @throws MalformedURLException if URL is malformed
    +     * @throws IOException in case of connection failure
    +     */
    +    private JsonElement sendRestRequest(String query) throws ProtocolException, MalformedURLException, IOException {
    +        // Create URL object
    +        URL url;
    +        try {
    +            url = new URL(query);
    +        }
    +        catch (MalformedURLException e) {
    +            LOG.error("sendRestRequest(): Failed (malformed URL). URL is '" + query + "'");
    --- End diff --
    
    there is usually no need to log the exception if you re-throw it, as the upstream logic should log it when it stops propagating it, otherwise the same problem would have multiple error log entries


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171117617
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteResolver.java ---
    @@ -0,0 +1,173 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +
    +import java.util.ArrayList;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.text.SimpleDateFormat;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.codec.binary.Base64;
    +import org.apache.commons.codec.binary.Hex;
    +
    +import com.google.gson.JsonArray;
    +
    +
    +
    +/**
    + * PXF-Ignite resolver class
    + */
    +public class IgniteResolver extends IgnitePlugin implements ReadResolver, WriteResolver {
    +    private static final Log LOG = LogFactory.getLog(IgniteResolver.class);
    +    
    +    // HAWQ column descriptors
    +    private ArrayList<ColumnDescriptor> columns = null;
    +
    +    public IgniteResolver(InputData input) throws Exception {
    +        super(input);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        columns = input.getTupleDescription();
    +        
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful");
    +        }
    +    }
    +
    +    /**
    +     * Transform a JsonArray object stored in {@link OneRow} into a List {@link OneField}
    +     */
    +    @Override
    +    public List<OneField> getFields(OneRow row) throws Exception {
    +        JsonArray result = (JsonArray)row.getData();
    +        LinkedList<OneField> fields = new LinkedList<OneField>();
    +
    +        for (int i = 0; i < columns.size(); i++) {
    --- End diff --
    
    what happens if the json array has more/less fields than the table? should we add a check that the sizes match?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171118261
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteResolver.java ---
    @@ -0,0 +1,173 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +
    +import java.util.ArrayList;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.text.SimpleDateFormat;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.codec.binary.Base64;
    +import org.apache.commons.codec.binary.Hex;
    +
    +import com.google.gson.JsonArray;
    +
    +
    +
    +/**
    + * PXF-Ignite resolver class
    + */
    +public class IgniteResolver extends IgnitePlugin implements ReadResolver, WriteResolver {
    +    private static final Log LOG = LogFactory.getLog(IgniteResolver.class);
    +    
    +    // HAWQ column descriptors
    +    private ArrayList<ColumnDescriptor> columns = null;
    +
    +    public IgniteResolver(InputData input) throws Exception {
    +        super(input);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        columns = input.getTupleDescription();
    +        
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful");
    +        }
    +    }
    +
    +    /**
    +     * Transform a JsonArray object stored in {@link OneRow} into a List {@link OneField}
    +     */
    +    @Override
    +    public List<OneField> getFields(OneRow row) throws Exception {
    +        JsonArray result = (JsonArray)row.getData();
    +        LinkedList<OneField> fields = new LinkedList<OneField>();
    +
    +        for (int i = 0; i < columns.size(); i++) {
    +            Object value = null;
    +            OneField oneField = new OneField(columns.get(i).columnTypeCode(), null);
    +
    +            // Handle null values
    +            if (result.get(i).isJsonNull()) {
    +                oneField.val = null;
    +                fields.add(oneField);
    +                continue;
    +            }
    +            switch (DataType.get(oneField.type)) {
    +                case INTEGER:
    +                    value = result.get(i).getAsInt();
    +                    break;
    +                case FLOAT8:
    +                    value = result.get(i).getAsDouble();
    +                    break;
    +                case REAL:
    +                    value = result.get(i).getAsFloat();
    +                    break;
    +                case BIGINT:
    +                    value = result.get(i).getAsLong();
    +                    break;
    +                case SMALLINT:
    +                    value = result.get(i).getAsShort();
    +                    break;
    +                case BOOLEAN:
    +                    value = result.get(i).getAsBoolean();
    +                    break;
    +                case VARCHAR:
    +                case BPCHAR:
    +                case TEXT:
    +                case NUMERIC:
    +                    value = result.get(i).getAsString();
    +                    break;
    +                case BYTEA:
    +                    value = Base64.decodeBase64(result.get(i).getAsString());
    +                    break;
    +                case TIMESTAMP:
    +                    value = new SimpleDateFormat("MMM d, yyyy hh:mm:ss a").parse(result.get(i).getAsString());
    +                    break;
    +                case DATE:
    +                    value = new SimpleDateFormat("yyyy-MM-dd").parse(result.get(i).getAsString());
    +                    break;
    +                default:
    +                    throw new UnsupportedOperationException("Field type not supported: " + DataType.get(oneField.type).toString()
    +                            + ", Column : " + columns.get(i).columnName());
    +            }
    +
    +            oneField.val = value;
    +            fields.add(oneField);
    +        }
    +
    +        return fields;
    +    }
    +
    +    /**
    +     * Create a query string from a List<OneField> for Ignite database REST API containing INSERT query
    +     */
    +    @Override
    +    public OneRow setFields(List<OneField> record) throws Exception {
    +        StringBuilder sb = new StringBuilder();
    +        String fieldDivisor = "";
    +        
    +        sb.append("(");
    +        for (OneField oneField : record) {
    +            sb.append(fieldDivisor);
    +            fieldDivisor = ", ";
    +            switch (DataType.get(oneField.type)) {
    +                case INTEGER:
    +                case FLOAT8:
    +                case REAL:
    +                case BIGINT:
    +                case SMALLINT:
    +                case BOOLEAN:
    +                    sb.append(String.valueOf(oneField.val));
    +                    break;
    +                case VARCHAR:
    +                case BPCHAR:
    +                case TEXT:
    +                case NUMERIC:
    +                    sb.append("'" + String.valueOf(oneField.val) + "'");
    +                    break;
    +                case BYTEA:
    +                    sb.append("'" + Hex.encodeHexString((byte[])(oneField.val)) + "'");
    --- End diff --
    
    usually getFields and setFields use the same logic (the idea is that if you write and then read the same data, you will get the same data back). Just wondering why use here Hex and not Base64 serialization?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171043644
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    +        return true;
    +    }
    +
    +    /**
    +     * readNextObject() implementation
    +     */
    +    @Override
    +    public OneRow readNextObject() throws Exception {
    +        if (urlReadFetch == null) {
    +            LOG.error("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +            throw new ProtocolException("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +        }
    +
    +        if (bufferRead.isEmpty()) {
    +            // Refill buffer
    +            if (isLastReadFinished) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +
    +            JsonElement response = sendRestRequest(urlReadFetch);
    +            isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +
    +            // Parse 'items'
    +            Iterator<JsonElement> itemsIterator = response.getAsJsonObject().get("items").getAsJsonArray().iterator();
    +            while (itemsIterator.hasNext()) {
    +                if (!bufferRead.add(itemsIterator.next().getAsJsonArray())) {
    +                    LOG.error("readNextObject(): Buffer refill failed (not enough memory in 'bufferRead')");
    +                    throw new OutOfMemoryError("readNextObject(): not enough memory in 'bufferRead'");
    --- End diff --
    
    I think you should use some application-level exception, OOME is usually reserved for JVM to complain when it can't get more memory.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171079170
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    +        return true;
    +    }
    +
    +    /**
    +     * readNextObject() implementation
    +     */
    +    @Override
    +    public OneRow readNextObject() throws Exception {
    +        if (urlReadFetch == null) {
    +            LOG.error("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +            throw new ProtocolException("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +        }
    +
    +        if (bufferRead.isEmpty()) {
    +            // Refill buffer
    +            if (isLastReadFinished) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +
    +            JsonElement response = sendRestRequest(urlReadFetch);
    +            isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +
    +            // Parse 'items'
    +            Iterator<JsonElement> itemsIterator = response.getAsJsonObject().get("items").getAsJsonArray().iterator();
    +            while (itemsIterator.hasNext()) {
    +                if (!bufferRead.add(itemsIterator.next().getAsJsonArray())) {
    +                    LOG.error("readNextObject(): Buffer refill failed (not enough memory in 'bufferRead')");
    +                    throw new OutOfMemoryError("readNextObject(): not enough memory in 'bufferRead'");
    +                }
    +            }
    +
    +            // Check again in case "response" contains no elements
    +            if (bufferRead.isEmpty()) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): Buffer refill failed");
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +        }
    +
    +        return new OneRow(bufferRead.pollFirst());
    +    }
    +
    +    /**
    +     * closeForRead() implementation
    +     */
    +    @Override
    +    public void closeForRead() {
    +        if (urlReadClose != null) {
    +            try {
    +                sendRestRequest(urlReadClose);
    +            }
    +            catch (Exception e) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("closeForRead() Exception: " + e.getClass().getSimpleName());
    +                }
    +            }
    +        }
    +        isLastReadFinished = false;
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Ignite read request finished. URL: '" + urlReadClose + "'");
    +        }
    +    }
    +
    +    /**
    +     * openForWrite() implementation.
    +     * No queries are sent to Ignite by this procedure, so if there are some problems (for example, with connection), they will be revealed only during the execution of 'writeNextObject()'
    +     */
    +    @Override
    +    public boolean openForWrite() throws UserDataException {
    +        // This is a temporary solution. At the moment there is no other way (except for the usage of user-defined parameters) to get the correct name of Ignite table: GPDB inserts extra data into the address, as required by Hadoop.
    +        // Note that if no extra data is present, the 'definedSource' will be left unchanged
    +        String definedSource = inputData.getDataSource();
    +        Pattern pattern = Pattern.compile("/(.*)/[0-9]*-[0-9]*_[0-9]*");
    +        Matcher matcher = pattern.matcher(definedSource);
    +        if (matcher.find()) {
    +            inputData.setDataSource(matcher.group(1));
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("INSERT INTO ");
    +
    +        // Insert the table name
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert the column names
    +        sb.append("(");
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        String fieldDivisor = "";
    +        for (int i = 0; i < columns.size(); i++) {
    --- End diff --
    
    this is very similar to the code in openForRead - consider using a helper function?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171118390
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/WhereSQLBuilder.java ---
    @@ -0,0 +1,140 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.LogicalFilter;
    +import org.apache.hawq.pxf.api.BasicFilter;
    +import org.apache.hawq.pxf.api.FilterParser;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +
    +/**
    + * Parse filter object generated by parent class  {@link org.apache.hawq.pxf.plugins.ignite.IgniteFilterBuilder},
    --- End diff --
    
    minor typos - please remove extra spaces: after `parent class`, after `Multiple filters`, after `HDOP_AND`, before and after `LogicalOperation`.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171117399
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteResolver.java ---
    @@ -0,0 +1,173 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +
    +import java.util.ArrayList;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.text.SimpleDateFormat;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.codec.binary.Base64;
    +import org.apache.commons.codec.binary.Hex;
    +
    +import com.google.gson.JsonArray;
    +
    +
    +
    +/**
    + * PXF-Ignite resolver class
    + */
    +public class IgniteResolver extends IgnitePlugin implements ReadResolver, WriteResolver {
    +    private static final Log LOG = LogFactory.getLog(IgniteResolver.class);
    +    
    +    // HAWQ column descriptors
    +    private ArrayList<ColumnDescriptor> columns = null;
    +
    +    public IgniteResolver(InputData input) throws Exception {
    +        super(input);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        columns = input.getTupleDescription();
    +        
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful");
    +        }
    +    }
    +
    +    /**
    +     * Transform a JsonArray object stored in {@link OneRow} into a List {@link OneField}
    +     */
    +    @Override
    +    public List<OneField> getFields(OneRow row) throws Exception {
    +        JsonArray result = (JsonArray)row.getData();
    +        LinkedList<OneField> fields = new LinkedList<OneField>();
    +
    +        for (int i = 0; i < columns.size(); i++) {
    +            Object value = null;
    +            OneField oneField = new OneField(columns.get(i).columnTypeCode(), null);
    +
    +            // Handle null values
    +            if (result.get(i).isJsonNull()) {
    +                oneField.val = null;
    --- End diff --
    
    is this line necessary? don't we initialize the oneField with null value in line 76?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171948500
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,411 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +
    +/**
    + * PXF-Ignite fragmenter class
    + * 
    + * This fragmenter works just like the one in PXF JDBC plugin
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    /**
    +     * Class constructor
    +     * 
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inputData) throws UserDataException {
    +        super(inputData);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inputData.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inputData.getUserProperty("PARTITION_BY").split(":");
    +            partitionType = PartitionType.typeOf(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' is invalid. The pattern is 'column_name:DATE|INT|ENUM'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inputData.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' is invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inputData.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.typeOf(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    +                }
    +            } 
    +            else if (partitionType != PartitionType.ENUM) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be specified along with 'PARTITION_BY'");
    +            }
    +            if (intervalNum < 1) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be at least 1. The actual is '" + intervalNum + "'");
    +            }
    +        } 
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'INTERVAL' invalid. The pattern is 'interval_num[:interval_unit]'");
    +        }
    +
    +        //parse date values
    +        try {
    +            if (partitionType == PartitionType.DATE) {
    +                SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd");
    +                rangeStart = Calendar.getInstance();
    +                rangeStart.setTime(df.parse(range[0]));
    +                rangeEnd = Calendar.getInstance();
    +                rangeEnd.setTime(df.parse(range[1]));
    +            }
    +        } catch (ParseException e) {
    +            throw new UserDataException("The parameter 'RANGE' has invalid date format. Expected format is 'yyyy-MM-dd'");
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful; some partition used");
    +        }
    +    }
    +
    +    /**
    +     * Returns statistics for the Ignite table. This is not implemented in the current version
    +     * @throws UnsupportedOperationException
    +     */
    +    @Override
    +    public FragmentsStats getFragmentsStats() throws UnsupportedOperationException {
    +        throw new UnsupportedOperationException("ANALYZE for Ignite plugin is not supported");
    +    }
    +
    +    /**
    +     * Returns list of fragments for Ignite table queries
    +     *
    +     * @throws UnsupportedOperationException if a partition of unknown type was found
    +     * 
    +     * @return a list of fragments
    +     */
    +    @Override
    +    public List<Fragment> getFragments() throws UnsupportedOperationException {
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() called; dataSource is '" + inputData.getDataSource() + "'");
    +        }
    +
    +        byte[] fragmentMetadata = null;
    +        byte[] fragmentUserdata = null;
    +
    +        if (partitionType == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() found no partition");
    +            }
    +            Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +            fragments.add(fragment);
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() successful");
    +            }
    +            return fragments;
    +        }
    +
    +        switch (partitionType) {
    +            case DATE: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found DATE partition");
    +                }
    +                int currInterval = intervalNum;
    +
    +                Calendar fragStart = rangeStart;
    +                while (fragStart.before(rangeEnd)) {
    +                    Calendar fragEnd = (Calendar) fragStart.clone();
    +                    switch (intervalType) {
    +                        case DAY:
    +                            fragEnd.add(Calendar.DAY_OF_MONTH, currInterval);
    +                            break;
    +                        case MONTH:
    +                            fragEnd.add(Calendar.MONTH, currInterval);
    +                            break;
    +                        case YEAR:
    +                            fragEnd.add(Calendar.YEAR, currInterval);
    +                            break;
    +                    }
    +                    if (fragEnd.after(rangeEnd))
    +                        fragEnd = (Calendar) rangeEnd.clone();
    +
    +                    // Note that the date is stored in milliseconds
    +                    byte[] msStart = ByteUtil.getBytes(fragStart.getTimeInMillis());
    +                    byte[] msEnd = ByteUtil.getBytes(fragEnd.getTimeInMillis());
    +                    fragmentMetadata = ArrayUtils.addAll(msStart, msEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;
    +                }
    +                break;
    +            }
    +            case INT: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found INT partition");
    +                }
    +                int rangeStart = Integer.parseInt(range[0]);
    +                int rangeEnd = Integer.parseInt(range[1]);
    +                int currInterval = intervalNum;
    +
    +                int fragStart = rangeStart;
    +                while (fragStart < rangeEnd) {
    +                    int fragEnd = fragStart + currInterval;
    +                    if (fragEnd > rangeEnd) {
    +                        fragEnd = rangeEnd;
    +                    }
    +
    +                    byte[] bStart = ByteUtil.getBytes(fragStart);
    +                    byte[] bEnd = ByteUtil.getBytes(fragEnd);
    +                    fragmentMetadata = ArrayUtils.addAll(bStart, bEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;
    +                }
    +                break;
    +            }
    +            case ENUM: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found ENUM partition");
    +                }
    +                for (String frag : range) {
    +                    fragmentMetadata = frag.getBytes();
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +                }
    +                break;
    +            }
    +            default: {
    +                throw new UnsupportedOperationException("getFragments() found a partition of unknown type and failed");
    +            }
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() successful");
    +        }
    +        return fragments;
    +    }
    +
    +    /**
    +     * Insert partition constraints into the prepared SQL query.
    +     * 
    +     * @param inputData pre-validated PXF InputData
    +     * @param sb the SQL query that is prepared for appending WHERE constraints.
    +     * Other SQL statements may be present, but they must be complete. Note that no check is performed to check their "completeness"
    +     */
    +    public static StringBuilder buildFragmenterSql(InputData inputData, StringBuilder sb) {
    --- End diff --
    
    Thank you! This is a better solution. Done.


---

[GitHub] incubator-hawq issue #1344: HAWQ-1599. PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344
  
    @kapustor -- we are ready to commit it if you do not plan any further changes. Need to put it through a regression run on our side, hope to be done with it next week. Thanks.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171130804
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,384 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +/**
    + * Fragmenter class for Ignite data resources.
    + * @note At the moment this fragmenter works just like the JDBC's one
    + *
    + * Aside from implementing PXF {@link Fragmenter},
    + * this class also transforms input data path (given by user) by adding the
    + * information on splitting the request into a few requests for different regions.
    + * <br>
    + * The parameter Patterns
    + * <br>
    + * There are three  parameters, the format is as follows:
    + * <br>
    + * <pre>
    + * <code>PARTITION_BY=column_name:column_type&amp;RANGE=start_value[:end_value]&amp;INTERVAL=interval_num[:interval_unit]</code>
    + * </pre>
    + * The <code>PARTITION_BY</code> parameter can be split by colon(':'),the <code>column_type</code> current supported : <code>date,int,enum</code> .
    + * The Date format is 'yyyy-MM-dd'. <br>
    + * The <code>RANGE</code> parameter can be split by colon(':') ,used to identify the starting range of each fragment.
    + * The range is left-closed, ie:<code> '&gt;= start_value AND &lt; end_value' </code>.If the <code>column_type</code> is <code>int</code>,
    + * the <code>end_value</code> can be empty. If the <code>column_type</code>is <code>enum</code>,the parameter <code>RANGE</code> can be empty. <br>
    + * The <code>INTERVAL</code> parameter can be split by colon(':'), indicate the interval value of one fragment.
    + * When <code>column_type</code> is <code>date</code>,this parameter must be split by colon, and <code>interval_unit</code> can be <code>year,month,day</code>.
    + * When <code>column_type</code> is <code>int</code>, the <code>interval_unit</code> can be empty.
    + * When <code>column_type</code> is <code>enum</code>,the <code>INTERVAL</code> parameter can be empty.
    + * <br>
    + * <p>
    + * The syntax examples is:
    + * <br>
    + * <code>PARTITION_BY=createdate:date&amp;RANGE=2008-01-01:2010-01-01&amp;INTERVAL=1:month'</code>
    + * <br>
    + * <code>PARTITION_BY=year:int&amp;RANGE=2008:2010&amp;INTERVAL=1</code>
    + * <br>
    + * <code>PARTITION_BY=grade:enum&amp;RANGE=excellent:good:general:bad</code>
    + * </p>
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    Log LOG = LogFactory.getLog(IgnitePartitionFragmenter.class);
    +
    +    String[] partitionBy = null;
    +    String[] range = null;
    +    String[] interval = null;
    +    PartitionType partitionType = null;
    +    String partitionColumn = null;
    +    IntervalType intervalType = null;
    +    int intervalNum = 1;
    +
    +    //when partitionType is DATE,it is valid
    +    Calendar rangeStart = null;
    +    Calendar rangeEnd = null;
    +
    +
    +    enum PartitionType {
    +        DATE,
    +        INT,
    +        ENUM;
    +
    +        public static PartitionType getType(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    enum IntervalType {
    +        DAY,
    +        MONTH,
    +        YEAR;
    +
    +        public static IntervalType type(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    /**
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inConf) throws UserDataException {
    +        super(inConf);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inConf.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inConf.getUserProperty("PARTITION_BY").split(":");
    +            partitionColumn = partitionBy[0];
    +            partitionType = PartitionType.getType(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' invalid, the pattern is 'column_name:date|int|enum'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inConf.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inConf.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.type(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    +                }
    +            } 
    +            else if (partitionType != PartitionType.ENUM) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be specified along with 'PARTITION_BY'");
    +            }
    +            if (intervalNum < 1) {
    +                throw new UserDataException("The parameter 'INTERVAL' must > 1, but actual is '" + intervalNum + "'");
    +            }
    +        } 
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'INTERVAL' invalid, the pattern is 'interval_num[:interval_unit]'");
    +        }
    +
    +        //parse any date values
    +        try {
    +            if (partitionType == PartitionType.DATE) {
    +                SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd");
    +                rangeStart = Calendar.getInstance();
    +                rangeStart.setTime(df.parse(range[0]));
    +                rangeEnd = Calendar.getInstance();
    +                rangeEnd.setTime(df.parse(range[1]));
    +            }
    +        } catch (ParseException e) {
    +            throw new UserDataException("The parameter 'RANGE' has invalid date format. Expected format is 'YYYY-MM-DD'");
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful; some partition used");
    +        }
    +    }
    +
    +    /**
    +     * Returns statistics for  Ignite table. Currently it's not implemented.
    +     * @throws UnsupportedOperationException
    +     */
    +    @Override
    +    public FragmentsStats getFragmentsStats() throws UnsupportedOperationException {
    +        throw new UnsupportedOperationException("ANALYZE for Ignite plugin is not supported");
    +    }
    +
    +    /**
    +     * Returns list of fragments for Ignite table queries
    +     *
    +     * @return a list of fragments
    +     * @throws UnknownHostException from java.net.InetAddress.getLocalHost().getHostAddress()
    +     */
    +    @Override
    +    public List<Fragment> getFragments() throws UnknownHostException {
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() called; dataSource is '" + inputData.getDataSource() + "'");
    +        }
    +
    +        // Always use 'localhost' as a replica holder
    +        // The replica holder is a *PXF* host which can process the requested query
    +        // Note that this is not an Ignite host
    +        String[] replicaHostAddressWrapped = new String[]{InetAddress.getLocalHost().getHostAddress()};
    +        byte[] fragmentMetadata = null;
    +        byte[] fragmentUserdata = null;
    +
    +        if (partitionType == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() found no partition");
    +            }
    +            Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +            fragments.add(fragment);
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() successful. Total fragments: " + fragments.size());
    +            }
    +            return fragments;
    +        }
    +
    +        switch (partitionType) {
    +            case DATE: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found DATE partition");
    +                }
    +                int currInterval = intervalNum;
    +
    +                Calendar fragStart = rangeStart;
    +                while (fragStart.before(rangeEnd)) {
    +                    Calendar fragEnd = (Calendar) fragStart.clone();
    +                    switch (intervalType) {
    +                        case DAY:
    +                            fragEnd.add(Calendar.DAY_OF_MONTH, currInterval);
    +                            break;
    +                        case MONTH:
    +                            fragEnd.add(Calendar.MONTH, currInterval);
    +                            break;
    +                        case YEAR:
    +                            fragEnd.add(Calendar.YEAR, currInterval);
    +                            break;
    +                    }
    +                    if (fragEnd.after(rangeEnd))
    +                        fragEnd = (Calendar) rangeEnd.clone();
    +
    +                    // Build Fragment.metadata: convert the date to a millisecond, then get bytes.
    +                    byte[] msStart = ByteUtil.getBytes(fragStart.getTimeInMillis());
    +                    byte[] msEnd = ByteUtil.getBytes(fragEnd.getTimeInMillis());
    +                    fragmentMetadata = ArrayUtils.addAll(msStart, msEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;
    +                }
    +                break;
    +            }
    +            case INT: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found INT partition");
    +                }
    +                int rangeStart = Integer.parseInt(range[0]);
    +                int rangeEnd = Integer.parseInt(range[1]);
    +                int currInterval = intervalNum;
    +
    +                // Validate : curr_interval > 0
    +                int fragStart = rangeStart;
    +                while (fragStart < rangeEnd) {
    +                    int fragEnd = fragStart + currInterval;
    +                    if (fragEnd > rangeEnd) fragEnd = rangeEnd;
    +
    +                    byte[] bStart = ByteUtil.getBytes(fragStart);
    +                    byte[] bEnd = ByteUtil.getBytes(fragEnd);
    +                    fragmentMetadata = ArrayUtils.addAll(bStart, bEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;// + 1;
    +                }
    +                break;
    +            }
    +            case ENUM:
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found ENUM partition");
    +                }
    +                for (String frag : range) {
    +                    fragmentMetadata = frag.getBytes();
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +                }
    +                break;
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() successful. Total fragments: " + fragments.size());
    +        }
    +        return fragments;
    +    }
    +
    +    public StringBuilder buildFragmenterSql(StringBuilder sb) {
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("buildFragmenterSql() called");
    +        }
    +
    +        byte[] meta = inputData.getFragmentMetadata();
    +        if (meta == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Fragment metadata is null; No partition constraints added");
    +            }
    +            return sb;
    +        }
    +
    +        if (!sb.toString().contains("WHERE")) {
    +            sb.append(" WHERE 1=1 ");
    --- End diff --
    
    can you please add a comment explaining why it is needed?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171117955
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteResolver.java ---
    @@ -0,0 +1,173 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +
    +import java.util.ArrayList;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.text.SimpleDateFormat;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.codec.binary.Base64;
    +import org.apache.commons.codec.binary.Hex;
    +
    +import com.google.gson.JsonArray;
    +
    +
    +
    +/**
    + * PXF-Ignite resolver class
    + */
    +public class IgniteResolver extends IgnitePlugin implements ReadResolver, WriteResolver {
    +    private static final Log LOG = LogFactory.getLog(IgniteResolver.class);
    +    
    +    // HAWQ column descriptors
    +    private ArrayList<ColumnDescriptor> columns = null;
    +
    +    public IgniteResolver(InputData input) throws Exception {
    +        super(input);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        columns = input.getTupleDescription();
    +        
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful");
    +        }
    +    }
    +
    +    /**
    +     * Transform a JsonArray object stored in {@link OneRow} into a List {@link OneField}
    +     */
    +    @Override
    +    public List<OneField> getFields(OneRow row) throws Exception {
    +        JsonArray result = (JsonArray)row.getData();
    +        LinkedList<OneField> fields = new LinkedList<OneField>();
    +
    +        for (int i = 0; i < columns.size(); i++) {
    +            Object value = null;
    +            OneField oneField = new OneField(columns.get(i).columnTypeCode(), null);
    +
    +            // Handle null values
    +            if (result.get(i).isJsonNull()) {
    +                oneField.val = null;
    +                fields.add(oneField);
    +                continue;
    +            }
    +            switch (DataType.get(oneField.type)) {
    +                case INTEGER:
    +                    value = result.get(i).getAsInt();
    +                    break;
    +                case FLOAT8:
    +                    value = result.get(i).getAsDouble();
    +                    break;
    +                case REAL:
    +                    value = result.get(i).getAsFloat();
    +                    break;
    +                case BIGINT:
    +                    value = result.get(i).getAsLong();
    +                    break;
    +                case SMALLINT:
    +                    value = result.get(i).getAsShort();
    +                    break;
    +                case BOOLEAN:
    +                    value = result.get(i).getAsBoolean();
    +                    break;
    +                case VARCHAR:
    +                case BPCHAR:
    +                case TEXT:
    +                case NUMERIC:
    +                    value = result.get(i).getAsString();
    +                    break;
    +                case BYTEA:
    +                    value = Base64.decodeBase64(result.get(i).getAsString());
    +                    break;
    +                case TIMESTAMP:
    +                    value = new SimpleDateFormat("MMM d, yyyy hh:mm:ss a").parse(result.get(i).getAsString());
    +                    break;
    +                case DATE:
    +                    value = new SimpleDateFormat("yyyy-MM-dd").parse(result.get(i).getAsString());
    +                    break;
    +                default:
    +                    throw new UnsupportedOperationException("Field type not supported: " + DataType.get(oneField.type).toString()
    +                            + ", Column : " + columns.get(i).columnName());
    +            }
    +
    +            oneField.val = value;
    +            fields.add(oneField);
    +        }
    +
    +        return fields;
    +    }
    +
    +    /**
    +     * Create a query string from a List<OneField> for Ignite database REST API containing INSERT query
    +     */
    +    @Override
    +    public OneRow setFields(List<OneField> record) throws Exception {
    +        StringBuilder sb = new StringBuilder();
    +        String fieldDivisor = "";
    +        
    +        sb.append("(");
    +        for (OneField oneField : record) {
    +            sb.append(fieldDivisor);
    +            fieldDivisor = ", ";
    +            switch (DataType.get(oneField.type)) {
    +                case INTEGER:
    +                case FLOAT8:
    +                case REAL:
    +                case BIGINT:
    +                case SMALLINT:
    +                case BOOLEAN:
    +                    sb.append(String.valueOf(oneField.val));
    +                    break;
    +                case VARCHAR:
    +                case BPCHAR:
    +                case TEXT:
    +                case NUMERIC:
    --- End diff --
    
    why does NUMERIC type need quotes? 


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171130229
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,384 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +/**
    + * Fragmenter class for Ignite data resources.
    + * @note At the moment this fragmenter works just like the JDBC's one
    + *
    + * Aside from implementing PXF {@link Fragmenter},
    + * this class also transforms input data path (given by user) by adding the
    + * information on splitting the request into a few requests for different regions.
    + * <br>
    + * The parameter Patterns
    + * <br>
    + * There are three  parameters, the format is as follows:
    + * <br>
    + * <pre>
    + * <code>PARTITION_BY=column_name:column_type&amp;RANGE=start_value[:end_value]&amp;INTERVAL=interval_num[:interval_unit]</code>
    + * </pre>
    + * The <code>PARTITION_BY</code> parameter can be split by colon(':'),the <code>column_type</code> current supported : <code>date,int,enum</code> .
    + * The Date format is 'yyyy-MM-dd'. <br>
    + * The <code>RANGE</code> parameter can be split by colon(':') ,used to identify the starting range of each fragment.
    + * The range is left-closed, ie:<code> '&gt;= start_value AND &lt; end_value' </code>.If the <code>column_type</code> is <code>int</code>,
    + * the <code>end_value</code> can be empty. If the <code>column_type</code>is <code>enum</code>,the parameter <code>RANGE</code> can be empty. <br>
    + * The <code>INTERVAL</code> parameter can be split by colon(':'), indicate the interval value of one fragment.
    + * When <code>column_type</code> is <code>date</code>,this parameter must be split by colon, and <code>interval_unit</code> can be <code>year,month,day</code>.
    + * When <code>column_type</code> is <code>int</code>, the <code>interval_unit</code> can be empty.
    + * When <code>column_type</code> is <code>enum</code>,the <code>INTERVAL</code> parameter can be empty.
    + * <br>
    + * <p>
    + * The syntax examples is:
    + * <br>
    + * <code>PARTITION_BY=createdate:date&amp;RANGE=2008-01-01:2010-01-01&amp;INTERVAL=1:month'</code>
    + * <br>
    + * <code>PARTITION_BY=year:int&amp;RANGE=2008:2010&amp;INTERVAL=1</code>
    + * <br>
    + * <code>PARTITION_BY=grade:enum&amp;RANGE=excellent:good:general:bad</code>
    + * </p>
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    Log LOG = LogFactory.getLog(IgnitePartitionFragmenter.class);
    +
    +    String[] partitionBy = null;
    +    String[] range = null;
    +    String[] interval = null;
    +    PartitionType partitionType = null;
    +    String partitionColumn = null;
    +    IntervalType intervalType = null;
    +    int intervalNum = 1;
    +
    +    //when partitionType is DATE,it is valid
    +    Calendar rangeStart = null;
    +    Calendar rangeEnd = null;
    +
    +
    +    enum PartitionType {
    +        DATE,
    +        INT,
    +        ENUM;
    +
    +        public static PartitionType getType(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    enum IntervalType {
    +        DAY,
    +        MONTH,
    +        YEAR;
    +
    +        public static IntervalType type(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    /**
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inConf) throws UserDataException {
    +        super(inConf);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inConf.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inConf.getUserProperty("PARTITION_BY").split(":");
    +            partitionColumn = partitionBy[0];
    +            partitionType = PartitionType.getType(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' invalid, the pattern is 'column_name:date|int|enum'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inConf.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inConf.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.type(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    +                }
    +            } 
    +            else if (partitionType != PartitionType.ENUM) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be specified along with 'PARTITION_BY'");
    +            }
    +            if (intervalNum < 1) {
    +                throw new UserDataException("The parameter 'INTERVAL' must > 1, but actual is '" + intervalNum + "'");
    +            }
    +        } 
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'INTERVAL' invalid, the pattern is 'interval_num[:interval_unit]'");
    +        }
    +
    +        //parse any date values
    +        try {
    +            if (partitionType == PartitionType.DATE) {
    +                SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd");
    +                rangeStart = Calendar.getInstance();
    +                rangeStart.setTime(df.parse(range[0]));
    +                rangeEnd = Calendar.getInstance();
    +                rangeEnd.setTime(df.parse(range[1]));
    +            }
    +        } catch (ParseException e) {
    +            throw new UserDataException("The parameter 'RANGE' has invalid date format. Expected format is 'YYYY-MM-DD'");
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful; some partition used");
    +        }
    +    }
    +
    +    /**
    +     * Returns statistics for  Ignite table. Currently it's not implemented.
    +     * @throws UnsupportedOperationException
    +     */
    +    @Override
    +    public FragmentsStats getFragmentsStats() throws UnsupportedOperationException {
    +        throw new UnsupportedOperationException("ANALYZE for Ignite plugin is not supported");
    +    }
    +
    +    /**
    +     * Returns list of fragments for Ignite table queries
    +     *
    +     * @return a list of fragments
    +     * @throws UnknownHostException from java.net.InetAddress.getLocalHost().getHostAddress()
    +     */
    +    @Override
    +    public List<Fragment> getFragments() throws UnknownHostException {
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() called; dataSource is '" + inputData.getDataSource() + "'");
    +        }
    +
    +        // Always use 'localhost' as a replica holder
    +        // The replica holder is a *PXF* host which can process the requested query
    +        // Note that this is not an Ignite host
    +        String[] replicaHostAddressWrapped = new String[]{InetAddress.getLocalHost().getHostAddress()};
    +        byte[] fragmentMetadata = null;
    +        byte[] fragmentUserdata = null;
    +
    +        if (partitionType == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() found no partition");
    +            }
    +            Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +            fragments.add(fragment);
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() successful. Total fragments: " + fragments.size());
    --- End diff --
    
    minor - don't really need to calculate the size here, we know it's 1. 


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171145415
  
    --- Diff: pxf/pxf-ignite/src/test/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessorTest.java ---
    @@ -0,0 +1,130 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import com.google.gson.JsonArray;
    +import com.google.gson.JsonObject;
    +
    +import org.junit.Test;
    +import org.junit.Before;
    +import org.junit.runner.RunWith;
    +import static org.junit.Assert.*;
    +
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Mockito;
    +import static org.mockito.Matchers.anyString;
    +
    +import org.powermock.api.mockito.PowerMockito;
    +import org.powermock.core.classloader.annotations.PrepareForTest;
    +import org.powermock.modules.junit4.PowerMockRunner;
    +import static org.powermock.api.support.membermodification.MemberMatcher.method;
    +
    +
    +@RunWith(PowerMockRunner.class)
    --- End diff --
    
    Really good tests!


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r170696659
  
    --- Diff: pxf/pxf-ignite/README.md ---
    @@ -0,0 +1,65 @@
    +# Accessing Ignite database using PXF
    +
    +The PXF Ignite plug-in enables to access the [Ignite database](https://ignite.apache.org/) (both read and write operations are supported) via REST API.
    +
    +
    +# Prerequisites
    +
    +Check the following before using the plug-in:
    +
    +* The Ignite plug-in is installed on all PXF nodes;
    +
    +* The Ignite client is installed and running at the `IGNITE_HOST` (`localhost` by default; see below), and it accepts http queries from the PXF (note that *enabling Ignite REST API does not require changes in Ignite configuration*; see the instruction on how to do that at https://apacheignite.readme.io/docs/rest-api#section-getting-started).
    +
    +
    +# Syntax
    +
    +```
    +CREATE [READABLE] EXTERNAL TABLE <table_name> (
    +    <column_name> <data_type>[, <column_name> <data_type>, ...] | LIKE <other_table>
    +)
    +LOCATION ('pxf://<ignite_table_name>?PROFILE=Ignite[&<extra-parameter>&<extra-parameter>&...]')
    +FORMAT 'CUSTOM' (formatter='pxfwritable_import');
    +```
    +where each `<extra-parameter>` is one of the following:
    +* `IGNITE_HOST=<ignite_host_address_with_port>`. The location of Ignite client node. If not given, `127.0.0.1:8080` is used by default;
    +* `IGNITE_CACHE=<ignite_cache_name>`. The name of Ignite cache to use. If not given, this parameter is not included in queries from PXF to Ignite, thus Ignite default values will be used (at the moment, this is `Default` cache). This option is **case-sensitive**;
    +* `BUFFER_SIZE=<unsigned_int>`. The number of tuples send to (from) Ignite per a response. The same number of tuples is stored in in-plug-in cache. The values `0` and `1` are equal (cache is not used, each tuple is passed in it's own query to Ignite). If not given, `128` is used by default;
    +* `PARTITION_BY=<column>:<column_type>`. See below;
    +* `RANGE=<start_value>:<end_value>`. See below;
    +* `INTERVAL=<value>[:<unit>]`. See below.
    +
    +
    +# Partitioning
    +## Introduction
    +
    +PXF Ignite plugin supports simultaneous access to Ignite database from multiple PXF segments. *Partitioning* should be used in order to perform such operation.
    +
    +Partitioning in PXF Ignite plug-in works just like in PXF JDBC plug-in.
    +
    +This feature is optional. However, a bug in the `pxf-service` which makes partitioning necessary for any query was fixed only on 17th Jan 2018 in [this commit](https://github.com/apache/incubator-hawq/commit/0d620e431026834dd70c9e0d63edf8bb28b38227), so the older versions of PXF may return an exception if a query does not contain a meaningful `PARTITION_BY` parameter.
    --- End diff --
    
    What's default partitioning strategy if `PARTITION_BY` was omitted?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171130678
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,384 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +/**
    + * Fragmenter class for Ignite data resources.
    + * @note At the moment this fragmenter works just like the JDBC's one
    + *
    + * Aside from implementing PXF {@link Fragmenter},
    + * this class also transforms input data path (given by user) by adding the
    + * information on splitting the request into a few requests for different regions.
    + * <br>
    + * The parameter Patterns
    + * <br>
    + * There are three  parameters, the format is as follows:
    + * <br>
    + * <pre>
    + * <code>PARTITION_BY=column_name:column_type&amp;RANGE=start_value[:end_value]&amp;INTERVAL=interval_num[:interval_unit]</code>
    + * </pre>
    + * The <code>PARTITION_BY</code> parameter can be split by colon(':'),the <code>column_type</code> current supported : <code>date,int,enum</code> .
    + * The Date format is 'yyyy-MM-dd'. <br>
    + * The <code>RANGE</code> parameter can be split by colon(':') ,used to identify the starting range of each fragment.
    + * The range is left-closed, ie:<code> '&gt;= start_value AND &lt; end_value' </code>.If the <code>column_type</code> is <code>int</code>,
    + * the <code>end_value</code> can be empty. If the <code>column_type</code>is <code>enum</code>,the parameter <code>RANGE</code> can be empty. <br>
    + * The <code>INTERVAL</code> parameter can be split by colon(':'), indicate the interval value of one fragment.
    + * When <code>column_type</code> is <code>date</code>,this parameter must be split by colon, and <code>interval_unit</code> can be <code>year,month,day</code>.
    + * When <code>column_type</code> is <code>int</code>, the <code>interval_unit</code> can be empty.
    + * When <code>column_type</code> is <code>enum</code>,the <code>INTERVAL</code> parameter can be empty.
    + * <br>
    + * <p>
    + * The syntax examples is:
    + * <br>
    + * <code>PARTITION_BY=createdate:date&amp;RANGE=2008-01-01:2010-01-01&amp;INTERVAL=1:month'</code>
    + * <br>
    + * <code>PARTITION_BY=year:int&amp;RANGE=2008:2010&amp;INTERVAL=1</code>
    + * <br>
    + * <code>PARTITION_BY=grade:enum&amp;RANGE=excellent:good:general:bad</code>
    + * </p>
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    Log LOG = LogFactory.getLog(IgnitePartitionFragmenter.class);
    +
    +    String[] partitionBy = null;
    +    String[] range = null;
    +    String[] interval = null;
    +    PartitionType partitionType = null;
    +    String partitionColumn = null;
    +    IntervalType intervalType = null;
    +    int intervalNum = 1;
    +
    +    //when partitionType is DATE,it is valid
    +    Calendar rangeStart = null;
    +    Calendar rangeEnd = null;
    +
    +
    +    enum PartitionType {
    +        DATE,
    +        INT,
    +        ENUM;
    +
    +        public static PartitionType getType(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    enum IntervalType {
    +        DAY,
    +        MONTH,
    +        YEAR;
    +
    +        public static IntervalType type(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    /**
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inConf) throws UserDataException {
    +        super(inConf);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inConf.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inConf.getUserProperty("PARTITION_BY").split(":");
    +            partitionColumn = partitionBy[0];
    +            partitionType = PartitionType.getType(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' invalid, the pattern is 'column_name:date|int|enum'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inConf.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inConf.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.type(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    +                }
    +            } 
    +            else if (partitionType != PartitionType.ENUM) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be specified along with 'PARTITION_BY'");
    +            }
    +            if (intervalNum < 1) {
    +                throw new UserDataException("The parameter 'INTERVAL' must > 1, but actual is '" + intervalNum + "'");
    +            }
    +        } 
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'INTERVAL' invalid, the pattern is 'interval_num[:interval_unit]'");
    +        }
    +
    +        //parse any date values
    +        try {
    +            if (partitionType == PartitionType.DATE) {
    +                SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd");
    +                rangeStart = Calendar.getInstance();
    +                rangeStart.setTime(df.parse(range[0]));
    +                rangeEnd = Calendar.getInstance();
    +                rangeEnd.setTime(df.parse(range[1]));
    +            }
    +        } catch (ParseException e) {
    +            throw new UserDataException("The parameter 'RANGE' has invalid date format. Expected format is 'YYYY-MM-DD'");
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful; some partition used");
    +        }
    +    }
    +
    +    /**
    +     * Returns statistics for  Ignite table. Currently it's not implemented.
    +     * @throws UnsupportedOperationException
    +     */
    +    @Override
    +    public FragmentsStats getFragmentsStats() throws UnsupportedOperationException {
    +        throw new UnsupportedOperationException("ANALYZE for Ignite plugin is not supported");
    +    }
    +
    +    /**
    +     * Returns list of fragments for Ignite table queries
    +     *
    +     * @return a list of fragments
    +     * @throws UnknownHostException from java.net.InetAddress.getLocalHost().getHostAddress()
    +     */
    +    @Override
    +    public List<Fragment> getFragments() throws UnknownHostException {
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() called; dataSource is '" + inputData.getDataSource() + "'");
    +        }
    +
    +        // Always use 'localhost' as a replica holder
    +        // The replica holder is a *PXF* host which can process the requested query
    +        // Note that this is not an Ignite host
    +        String[] replicaHostAddressWrapped = new String[]{InetAddress.getLocalHost().getHostAddress()};
    +        byte[] fragmentMetadata = null;
    +        byte[] fragmentUserdata = null;
    +
    +        if (partitionType == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() found no partition");
    +            }
    +            Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +            fragments.add(fragment);
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("getFragments() successful. Total fragments: " + fragments.size());
    +            }
    +            return fragments;
    +        }
    +
    +        switch (partitionType) {
    +            case DATE: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found DATE partition");
    +                }
    +                int currInterval = intervalNum;
    +
    +                Calendar fragStart = rangeStart;
    +                while (fragStart.before(rangeEnd)) {
    +                    Calendar fragEnd = (Calendar) fragStart.clone();
    +                    switch (intervalType) {
    +                        case DAY:
    +                            fragEnd.add(Calendar.DAY_OF_MONTH, currInterval);
    +                            break;
    +                        case MONTH:
    +                            fragEnd.add(Calendar.MONTH, currInterval);
    +                            break;
    +                        case YEAR:
    +                            fragEnd.add(Calendar.YEAR, currInterval);
    +                            break;
    +                    }
    +                    if (fragEnd.after(rangeEnd))
    +                        fragEnd = (Calendar) rangeEnd.clone();
    +
    +                    // Build Fragment.metadata: convert the date to a millisecond, then get bytes.
    +                    byte[] msStart = ByteUtil.getBytes(fragStart.getTimeInMillis());
    +                    byte[] msEnd = ByteUtil.getBytes(fragEnd.getTimeInMillis());
    +                    fragmentMetadata = ArrayUtils.addAll(msStart, msEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;
    +                }
    +                break;
    +            }
    +            case INT: {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("getFragments() found INT partition");
    +                }
    +                int rangeStart = Integer.parseInt(range[0]);
    +                int rangeEnd = Integer.parseInt(range[1]);
    +                int currInterval = intervalNum;
    +
    +                // Validate : curr_interval > 0
    +                int fragStart = rangeStart;
    +                while (fragStart < rangeEnd) {
    +                    int fragEnd = fragStart + currInterval;
    +                    if (fragEnd > rangeEnd) fragEnd = rangeEnd;
    +
    +                    byte[] bStart = ByteUtil.getBytes(fragStart);
    +                    byte[] bEnd = ByteUtil.getBytes(fragEnd);
    +                    fragmentMetadata = ArrayUtils.addAll(bStart, bEnd);
    +
    +                    Fragment fragment = new Fragment(inputData.getDataSource(), replicaHostAddressWrapped, fragmentMetadata, fragmentUserdata);
    +                    fragments.add(fragment);
    +
    +                    // Continue the previous fragment
    +                    fragStart = fragEnd;// + 1;
    --- End diff --
    
    remove comment?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171131448
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    --- End diff --
    
    typo constraints


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171129628
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,384 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +/**
    + * Fragmenter class for Ignite data resources.
    + * @note At the moment this fragmenter works just like the JDBC's one
    + *
    + * Aside from implementing PXF {@link Fragmenter},
    + * this class also transforms input data path (given by user) by adding the
    + * information on splitting the request into a few requests for different regions.
    + * <br>
    + * The parameter Patterns
    + * <br>
    + * There are three  parameters, the format is as follows:
    + * <br>
    + * <pre>
    + * <code>PARTITION_BY=column_name:column_type&amp;RANGE=start_value[:end_value]&amp;INTERVAL=interval_num[:interval_unit]</code>
    + * </pre>
    + * The <code>PARTITION_BY</code> parameter can be split by colon(':'),the <code>column_type</code> current supported : <code>date,int,enum</code> .
    + * The Date format is 'yyyy-MM-dd'. <br>
    + * The <code>RANGE</code> parameter can be split by colon(':') ,used to identify the starting range of each fragment.
    + * The range is left-closed, ie:<code> '&gt;= start_value AND &lt; end_value' </code>.If the <code>column_type</code> is <code>int</code>,
    + * the <code>end_value</code> can be empty. If the <code>column_type</code>is <code>enum</code>,the parameter <code>RANGE</code> can be empty. <br>
    + * The <code>INTERVAL</code> parameter can be split by colon(':'), indicate the interval value of one fragment.
    + * When <code>column_type</code> is <code>date</code>,this parameter must be split by colon, and <code>interval_unit</code> can be <code>year,month,day</code>.
    + * When <code>column_type</code> is <code>int</code>, the <code>interval_unit</code> can be empty.
    + * When <code>column_type</code> is <code>enum</code>,the <code>INTERVAL</code> parameter can be empty.
    + * <br>
    + * <p>
    + * The syntax examples is:
    + * <br>
    + * <code>PARTITION_BY=createdate:date&amp;RANGE=2008-01-01:2010-01-01&amp;INTERVAL=1:month'</code>
    + * <br>
    + * <code>PARTITION_BY=year:int&amp;RANGE=2008:2010&amp;INTERVAL=1</code>
    + * <br>
    + * <code>PARTITION_BY=grade:enum&amp;RANGE=excellent:good:general:bad</code>
    + * </p>
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    Log LOG = LogFactory.getLog(IgnitePartitionFragmenter.class);
    +
    +    String[] partitionBy = null;
    +    String[] range = null;
    +    String[] interval = null;
    +    PartitionType partitionType = null;
    +    String partitionColumn = null;
    +    IntervalType intervalType = null;
    +    int intervalNum = 1;
    +
    +    //when partitionType is DATE,it is valid
    +    Calendar rangeStart = null;
    +    Calendar rangeEnd = null;
    +
    +
    +    enum PartitionType {
    +        DATE,
    +        INT,
    +        ENUM;
    +
    +        public static PartitionType getType(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    enum IntervalType {
    +        DAY,
    +        MONTH,
    +        YEAR;
    +
    +        public static IntervalType type(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    /**
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inConf) throws UserDataException {
    +        super(inConf);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inConf.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inConf.getUserProperty("PARTITION_BY").split(":");
    +            partitionColumn = partitionBy[0];
    +            partitionType = PartitionType.getType(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' invalid, the pattern is 'column_name:date|int|enum'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inConf.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' invalid, the pattern is 'start_value[:end_value]'");
    --- End diff --
    
    looks like the pattern is `start_value[[:enum_values]*:end_value]` or something like that.
    Maybe write: `The parameter 'RANGE' is invalid, the pattern is 'start_value[:end_value]' or 'enum_value1:enum_value2:...enum_valueN'`?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171130514
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,384 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +/**
    + * Fragmenter class for Ignite data resources.
    + * @note At the moment this fragmenter works just like the JDBC's one
    + *
    + * Aside from implementing PXF {@link Fragmenter},
    + * this class also transforms input data path (given by user) by adding the
    + * information on splitting the request into a few requests for different regions.
    + * <br>
    + * The parameter Patterns
    + * <br>
    + * There are three  parameters, the format is as follows:
    + * <br>
    + * <pre>
    + * <code>PARTITION_BY=column_name:column_type&amp;RANGE=start_value[:end_value]&amp;INTERVAL=interval_num[:interval_unit]</code>
    + * </pre>
    + * The <code>PARTITION_BY</code> parameter can be split by colon(':'),the <code>column_type</code> current supported : <code>date,int,enum</code> .
    + * The Date format is 'yyyy-MM-dd'. <br>
    + * The <code>RANGE</code> parameter can be split by colon(':') ,used to identify the starting range of each fragment.
    + * The range is left-closed, ie:<code> '&gt;= start_value AND &lt; end_value' </code>.If the <code>column_type</code> is <code>int</code>,
    + * the <code>end_value</code> can be empty. If the <code>column_type</code>is <code>enum</code>,the parameter <code>RANGE</code> can be empty. <br>
    + * The <code>INTERVAL</code> parameter can be split by colon(':'), indicate the interval value of one fragment.
    + * When <code>column_type</code> is <code>date</code>,this parameter must be split by colon, and <code>interval_unit</code> can be <code>year,month,day</code>.
    + * When <code>column_type</code> is <code>int</code>, the <code>interval_unit</code> can be empty.
    + * When <code>column_type</code> is <code>enum</code>,the <code>INTERVAL</code> parameter can be empty.
    + * <br>
    + * <p>
    + * The syntax examples is:
    + * <br>
    + * <code>PARTITION_BY=createdate:date&amp;RANGE=2008-01-01:2010-01-01&amp;INTERVAL=1:month'</code>
    + * <br>
    + * <code>PARTITION_BY=year:int&amp;RANGE=2008:2010&amp;INTERVAL=1</code>
    + * <br>
    + * <code>PARTITION_BY=grade:enum&amp;RANGE=excellent:good:general:bad</code>
    + * </p>
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    Log LOG = LogFactory.getLog(IgnitePartitionFragmenter.class);
    +
    +    String[] partitionBy = null;
    +    String[] range = null;
    +    String[] interval = null;
    +    PartitionType partitionType = null;
    +    String partitionColumn = null;
    +    IntervalType intervalType = null;
    +    int intervalNum = 1;
    +
    +    //when partitionType is DATE,it is valid
    +    Calendar rangeStart = null;
    +    Calendar rangeEnd = null;
    +
    +
    +    enum PartitionType {
    +        DATE,
    +        INT,
    +        ENUM;
    +
    +        public static PartitionType getType(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    enum IntervalType {
    +        DAY,
    +        MONTH,
    +        YEAR;
    +
    +        public static IntervalType type(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    /**
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inConf) throws UserDataException {
    +        super(inConf);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inConf.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inConf.getUserProperty("PARTITION_BY").split(":");
    +            partitionColumn = partitionBy[0];
    +            partitionType = PartitionType.getType(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' invalid, the pattern is 'column_name:date|int|enum'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inConf.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inConf.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.type(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    +                }
    +            } 
    +            else if (partitionType != PartitionType.ENUM) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be specified along with 'PARTITION_BY'");
    +            }
    +            if (intervalNum < 1) {
    +                throw new UserDataException("The parameter 'INTERVAL' must > 1, but actual is '" + intervalNum + "'");
    +            }
    +        } 
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'INTERVAL' invalid, the pattern is 'interval_num[:interval_unit]'");
    +        }
    +
    +        //parse any date values
    +        try {
    +            if (partitionType == PartitionType.DATE) {
    +                SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd");
    +                rangeStart = Calendar.getInstance();
    +                rangeStart.setTime(df.parse(range[0]));
    +                rangeEnd = Calendar.getInstance();
    +                rangeEnd.setTime(df.parse(range[1]));
    +            }
    +        } catch (ParseException e) {
    +            throw new UserDataException("The parameter 'RANGE' has invalid date format. Expected format is 'YYYY-MM-DD'");
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful; some partition used");
    +        }
    +    }
    +
    +    /**
    +     * Returns statistics for  Ignite table. Currently it's not implemented.
    +     * @throws UnsupportedOperationException
    +     */
    +    @Override
    +    public FragmentsStats getFragmentsStats() throws UnsupportedOperationException {
    +        throw new UnsupportedOperationException("ANALYZE for Ignite plugin is not supported");
    +    }
    +
    +    /**
    +     * Returns list of fragments for Ignite table queries
    +     *
    +     * @return a list of fragments
    +     * @throws UnknownHostException from java.net.InetAddress.getLocalHost().getHostAddress()
    +     */
    +    @Override
    +    public List<Fragment> getFragments() throws UnknownHostException {
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("getFragments() called; dataSource is '" + inputData.getDataSource() + "'");
    +        }
    +
    +        // Always use 'localhost' as a replica holder
    +        // The replica holder is a *PXF* host which can process the requested query
    +        // Note that this is not an Ignite host
    +        String[] replicaHostAddressWrapped = new String[]{InetAddress.getLocalHost().getHostAddress()};
    +        byte[] fragmentMetadata = null;
    +        byte[] fragmentUserdata = null;
    --- End diff --
    
    since we never update it, maybe set it to final?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171129783
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,384 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +/**
    + * Fragmenter class for Ignite data resources.
    + * @note At the moment this fragmenter works just like the JDBC's one
    + *
    + * Aside from implementing PXF {@link Fragmenter},
    + * this class also transforms input data path (given by user) by adding the
    + * information on splitting the request into a few requests for different regions.
    + * <br>
    + * The parameter Patterns
    + * <br>
    + * There are three  parameters, the format is as follows:
    + * <br>
    + * <pre>
    + * <code>PARTITION_BY=column_name:column_type&amp;RANGE=start_value[:end_value]&amp;INTERVAL=interval_num[:interval_unit]</code>
    + * </pre>
    + * The <code>PARTITION_BY</code> parameter can be split by colon(':'),the <code>column_type</code> current supported : <code>date,int,enum</code> .
    + * The Date format is 'yyyy-MM-dd'. <br>
    + * The <code>RANGE</code> parameter can be split by colon(':') ,used to identify the starting range of each fragment.
    + * The range is left-closed, ie:<code> '&gt;= start_value AND &lt; end_value' </code>.If the <code>column_type</code> is <code>int</code>,
    + * the <code>end_value</code> can be empty. If the <code>column_type</code>is <code>enum</code>,the parameter <code>RANGE</code> can be empty. <br>
    + * The <code>INTERVAL</code> parameter can be split by colon(':'), indicate the interval value of one fragment.
    + * When <code>column_type</code> is <code>date</code>,this parameter must be split by colon, and <code>interval_unit</code> can be <code>year,month,day</code>.
    + * When <code>column_type</code> is <code>int</code>, the <code>interval_unit</code> can be empty.
    + * When <code>column_type</code> is <code>enum</code>,the <code>INTERVAL</code> parameter can be empty.
    + * <br>
    + * <p>
    + * The syntax examples is:
    + * <br>
    + * <code>PARTITION_BY=createdate:date&amp;RANGE=2008-01-01:2010-01-01&amp;INTERVAL=1:month'</code>
    + * <br>
    + * <code>PARTITION_BY=year:int&amp;RANGE=2008:2010&amp;INTERVAL=1</code>
    + * <br>
    + * <code>PARTITION_BY=grade:enum&amp;RANGE=excellent:good:general:bad</code>
    + * </p>
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    Log LOG = LogFactory.getLog(IgnitePartitionFragmenter.class);
    +
    +    String[] partitionBy = null;
    +    String[] range = null;
    +    String[] interval = null;
    +    PartitionType partitionType = null;
    +    String partitionColumn = null;
    +    IntervalType intervalType = null;
    +    int intervalNum = 1;
    +
    +    //when partitionType is DATE,it is valid
    +    Calendar rangeStart = null;
    +    Calendar rangeEnd = null;
    +
    +
    +    enum PartitionType {
    +        DATE,
    +        INT,
    +        ENUM;
    +
    +        public static PartitionType getType(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    enum IntervalType {
    +        DAY,
    +        MONTH,
    +        YEAR;
    +
    +        public static IntervalType type(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    /**
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inConf) throws UserDataException {
    +        super(inConf);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inConf.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inConf.getUserProperty("PARTITION_BY").split(":");
    +            partitionColumn = partitionBy[0];
    +            partitionType = PartitionType.getType(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' invalid, the pattern is 'column_name:date|int|enum'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inConf.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inConf.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.type(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    --- End diff --
    
    maybe change to `unit ':[year|month|day]`


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r170697245
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/ByteUtil.java ---
    @@ -0,0 +1,85 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +
    +import org.apache.commons.lang.ArrayUtils;
    +
    +/**
    + * A tool class, used to deal with byte array merging, split and other methods.
    + * IgnitePartitionFragmenter requires this class in order to work.
    + */
    +public class ByteUtil {
    +
    +    public static byte[] mergeBytes(byte[] b1, byte[] b2) {
    --- End diff --
    
    Why not use ArrayUtils.addAll directly?


---

[GitHub] incubator-hawq issue #1344: HAWQ-1599. PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344
  
    Hi guys,
    
    Any news on this PR?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171334359
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteResolver.java ---
    @@ -0,0 +1,173 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +
    +import java.util.ArrayList;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.text.SimpleDateFormat;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.codec.binary.Base64;
    +import org.apache.commons.codec.binary.Hex;
    +
    +import com.google.gson.JsonArray;
    +
    +
    +
    +/**
    + * PXF-Ignite resolver class
    + */
    +public class IgniteResolver extends IgnitePlugin implements ReadResolver, WriteResolver {
    +    private static final Log LOG = LogFactory.getLog(IgniteResolver.class);
    +    
    +    // HAWQ column descriptors
    +    private ArrayList<ColumnDescriptor> columns = null;
    +
    +    public IgniteResolver(InputData input) throws Exception {
    +        super(input);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        columns = input.getTupleDescription();
    +        
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful");
    +        }
    +    }
    +
    +    /**
    +     * Transform a JsonArray object stored in {@link OneRow} into a List {@link OneField}
    +     */
    +    @Override
    +    public List<OneField> getFields(OneRow row) throws Exception {
    +        JsonArray result = (JsonArray)row.getData();
    +        LinkedList<OneField> fields = new LinkedList<OneField>();
    +
    +        for (int i = 0; i < columns.size(); i++) {
    +            Object value = null;
    +            OneField oneField = new OneField(columns.get(i).columnTypeCode(), null);
    +
    +            // Handle null values
    +            if (result.get(i).isJsonNull()) {
    +                oneField.val = null;
    +                fields.add(oneField);
    +                continue;
    +            }
    +            switch (DataType.get(oneField.type)) {
    +                case INTEGER:
    +                    value = result.get(i).getAsInt();
    +                    break;
    +                case FLOAT8:
    +                    value = result.get(i).getAsDouble();
    +                    break;
    +                case REAL:
    +                    value = result.get(i).getAsFloat();
    +                    break;
    +                case BIGINT:
    +                    value = result.get(i).getAsLong();
    +                    break;
    +                case SMALLINT:
    +                    value = result.get(i).getAsShort();
    +                    break;
    +                case BOOLEAN:
    +                    value = result.get(i).getAsBoolean();
    +                    break;
    +                case VARCHAR:
    +                case BPCHAR:
    +                case TEXT:
    +                case NUMERIC:
    +                    value = result.get(i).getAsString();
    +                    break;
    +                case BYTEA:
    +                    value = Base64.decodeBase64(result.get(i).getAsString());
    +                    break;
    +                case TIMESTAMP:
    +                    value = new SimpleDateFormat("MMM d, yyyy hh:mm:ss a").parse(result.get(i).getAsString());
    +                    break;
    +                case DATE:
    +                    value = new SimpleDateFormat("yyyy-MM-dd").parse(result.get(i).getAsString());
    +                    break;
    +                default:
    +                    throw new UnsupportedOperationException("Field type not supported: " + DataType.get(oneField.type).toString()
    +                            + ", Column : " + columns.get(i).columnName());
    +            }
    +
    +            oneField.val = value;
    +            fields.add(oneField);
    +        }
    +
    +        return fields;
    +    }
    +
    +    /**
    +     * Create a query string from a List<OneField> for Ignite database REST API containing INSERT query
    +     */
    +    @Override
    +    public OneRow setFields(List<OneField> record) throws Exception {
    +        StringBuilder sb = new StringBuilder();
    +        String fieldDivisor = "";
    +        
    +        sb.append("(");
    +        for (OneField oneField : record) {
    +            sb.append(fieldDivisor);
    +            fieldDivisor = ", ";
    +            switch (DataType.get(oneField.type)) {
    +                case INTEGER:
    +                case FLOAT8:
    +                case REAL:
    +                case BIGINT:
    +                case SMALLINT:
    +                case BOOLEAN:
    +                    sb.append(String.valueOf(oneField.val));
    +                    break;
    +                case VARCHAR:
    +                case BPCHAR:
    +                case TEXT:
    +                case NUMERIC:
    +                    sb.append("'" + String.valueOf(oneField.val) + "'");
    +                    break;
    +                case BYTEA:
    +                    sb.append("'" + Hex.encodeHexString((byte[])(oneField.val)) + "'");
    --- End diff --
    
    That is how Ignite handles such fields :smile:
    
    I have not found the Ignite documentation for this case. However, when Base64 field is sent to Ignite, it handles it some wrong way (it does not return an error, too).


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171326726
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    +        return true;
    +    }
    +
    +    /**
    +     * readNextObject() implementation
    +     */
    +    @Override
    +    public OneRow readNextObject() throws Exception {
    +        if (urlReadFetch == null) {
    +            LOG.error("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +            throw new ProtocolException("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +        }
    +
    +        if (bufferRead.isEmpty()) {
    +            // Refill buffer
    +            if (isLastReadFinished) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +
    +            JsonElement response = sendRestRequest(urlReadFetch);
    +            isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +
    +            // Parse 'items'
    +            Iterator<JsonElement> itemsIterator = response.getAsJsonObject().get("items").getAsJsonArray().iterator();
    +            while (itemsIterator.hasNext()) {
    +                if (!bufferRead.add(itemsIterator.next().getAsJsonArray())) {
    +                    LOG.error("readNextObject(): Buffer refill failed (not enough memory in 'bufferRead')");
    +                    throw new OutOfMemoryError("readNextObject(): not enough memory in 'bufferRead'");
    +                }
    +            }
    +
    +            // Check again in case "response" contains no elements
    +            if (bufferRead.isEmpty()) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): Buffer refill failed");
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +        }
    +
    +        return new OneRow(bufferRead.pollFirst());
    +    }
    +
    +    /**
    +     * closeForRead() implementation
    +     */
    +    @Override
    +    public void closeForRead() {
    +        if (urlReadClose != null) {
    +            try {
    +                sendRestRequest(urlReadClose);
    +            }
    +            catch (Exception e) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("closeForRead() Exception: " + e.getClass().getSimpleName());
    +                }
    +            }
    +        }
    +        isLastReadFinished = false;
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Ignite read request finished. URL: '" + urlReadClose + "'");
    +        }
    +    }
    +
    +    /**
    +     * openForWrite() implementation.
    +     * No queries are sent to Ignite by this procedure, so if there are some problems (for example, with connection), they will be revealed only during the execution of 'writeNextObject()'
    +     */
    +    @Override
    +    public boolean openForWrite() throws UserDataException {
    +        // This is a temporary solution. At the moment there is no other way (except for the usage of user-defined parameters) to get the correct name of Ignite table: GPDB inserts extra data into the address, as required by Hadoop.
    +        // Note that if no extra data is present, the 'definedSource' will be left unchanged
    +        String definedSource = inputData.getDataSource();
    +        Pattern pattern = Pattern.compile("/(.*)/[0-9]*-[0-9]*_[0-9]*");
    +        Matcher matcher = pattern.matcher(definedSource);
    +        if (matcher.find()) {
    +            inputData.setDataSource(matcher.group(1));
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("INSERT INTO ");
    +
    +        // Insert the table name
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert the column names
    +        sb.append("(");
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        String fieldDivisor = "";
    +        for (int i = 0; i < columns.size(); i++) {
    --- End diff --
    
    I don't see the need for it here. Although it is used in both `openForRead()` and `openForWrite()`, in `openForWrite` it is surrounded by `(` and `)`, so the code there would look odd, I suppose. Besides, the function would encapsulate just six lines.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171132977
  
    --- Diff: pxf/pxf-ignite/src/test/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenterTest.java ---
    @@ -0,0 +1,268 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import java.util.Calendar;
    +import java.util.List;
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.ByteUtil;
    +
    +import org.junit.Before;
    +import org.junit.Test;
    +import static org.junit.Assert.*;
    +
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +public class IgnitePartitionFragmenterTest {
    +    InputData inputData;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        prepareConstruction();
    +        when(inputData.getDataSource()).thenReturn("sales");
    +    }
    +
    +    @Test
    +    public void testPartionByDateOfMonth() throws Exception {
    --- End diff --
    
    typo Partition (and in other test names below)


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171036634
  
    --- Diff: pxf/pxf-ignite/README.md ---
    @@ -0,0 +1,67 @@
    +# Accessing Ignite database using PXF
    +
    +The PXF Ignite plug-in enables to access the [Ignite database](https://ignite.apache.org/) (both read and write operations are supported) via REST API.
    +
    +
    +# Prerequisites
    +
    +Check the following before using the plug-in:
    +
    +* The Ignite plug-in is installed on all PXF nodes;
    +
    +* The Ignite client is installed and running at the `IGNITE_HOST` (`localhost` by default; see below), and it accepts http queries from the PXF (note that *enabling Ignite REST API does not require changes in Ignite configuration*; see the instruction on how to do that at https://apacheignite.readme.io/docs/rest-api#section-getting-started).
    +
    +
    +# Syntax
    +
    +```
    +CREATE [READABLE] EXTERNAL TABLE <table_name> (
    +    <column_name> <data_type>[, <column_name> <data_type>, ...] | LIKE <other_table>
    +)
    +LOCATION ('pxf://<ignite_table_name>?PROFILE=Ignite[&<extra-parameter>&<extra-parameter>&...]')
    +FORMAT 'CUSTOM' (formatter='pxfwritable_import');
    +```
    +where each `<extra-parameter>` is one of the following:
    +* `IGNITE_HOST=<ignite_host_address_with_port>`. The location of Ignite client node. If not given, `127.0.0.1:8080` is used by default;
    +* `IGNITE_CACHE=<ignite_cache_name>`. The name of Ignite cache to use. If not given, this parameter is not included in queries from PXF to Ignite, thus Ignite default values will be used (at the moment, this is `Default` cache). This option is **case-sensitive**;
    +* `BUFFER_SIZE=<unsigned_int>`. The number of tuples send to (from) Ignite per a response. The same number of tuples is stored in in-plug-in cache. The values `0` and `1` are equal (cache is not used, each tuple is passed in it's own query to Ignite). If not given, `128` is used by default;
    +* `PARTITION_BY=<column>:<column_type>`. See below;
    +* `RANGE=<start_value>:<end_value>`. See below;
    +* `INTERVAL=<value>[:<unit>]`. See below.
    +
    +
    +# Partitioning
    +## Introduction
    +
    +PXF Ignite plugin supports simultaneous access to Ignite database from multiple PXF segments. *Partitioning* should be used in order to perform such operation.
    +
    +If the partitioning is not used, all the data will be retrieved by a single PXF segment.
    +
    +Partitioning in PXF Ignite plug-in works just like in PXF JDBC plug-in.
    --- End diff --
    
    since this is a separate plugin and JDBC partitioning may change independently, it would be good to describe how it woks for Ignite here.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r170735616
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    --- End diff --
    
    This check is performed by `sendRestRequest()` function. If the request was not successfull, `sendRestRequest` will throw a `java.net.MalformedURLException`, `java.io.IOException` or `java.net.ProtocolException`. All these errors will include the URL that caused them and (in case of error reported by Ignite in it's response) additional data. As these exceptions should be passed "up", they are not handled here. See https://github.com/apache/incubator-hawq/pull/1344/files#diff-f5894bad29c44342eee2760e259a91d6R480 for the details.
    
    When the exception is thrown by `readNextObject()`, the procedure `closeForRead()` is called, as it would be in case `false` was returned. So both variants (`return false` and `throw ...`) are safe. However, (AFAIK) if the exception is not thrown by `readNextObject()`, the user will not get the message that the read went wrong. So throwing exceptions from `public` `IgniteAccessor` functions is necessary.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r170702646
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    --- End diff --
    
    Maybe check if a request was successful and return `false` if not.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r170730763
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/ByteUtil.java ---
    @@ -0,0 +1,85 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +
    +import org.apache.commons.lang.ArrayUtils;
    +
    +/**
    + * A tool class, used to deal with byte array merging, split and other methods.
    + * IgnitePartitionFragmenter requires this class in order to work.
    + */
    +public class ByteUtil {
    +
    +    public static byte[] mergeBytes(byte[] b1, byte[] b2) {
    --- End diff --
    
    This piece of code comes from PXF JDBC plugin; for sure, `ArrayUtils.addAll()` can be used directly; this also removes the only dependence of `ByteUtil.java`. As this package is used to process byte arrays in `IgnitePartitionFragmenter` anyway, an extra dependence may be imported there without any harm, I suppose.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171145340
  
    --- Diff: pxf/pxf-ignite/src/test/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessorTest.java ---
    @@ -0,0 +1,130 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import com.google.gson.JsonArray;
    +import com.google.gson.JsonObject;
    +
    +import org.junit.Test;
    +import org.junit.Before;
    +import org.junit.runner.RunWith;
    +import static org.junit.Assert.*;
    +
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Mockito;
    +import static org.mockito.Matchers.anyString;
    +
    +import org.powermock.api.mockito.PowerMockito;
    +import org.powermock.core.classloader.annotations.PrepareForTest;
    +import org.powermock.modules.junit4.PowerMockRunner;
    +import static org.powermock.api.support.membermodification.MemberMatcher.method;
    +
    +
    +@RunWith(PowerMockRunner.class)
    +@PrepareForTest({IgniteAccessor.class})
    +public class IgniteAccessorTest {
    +    @Before
    +    public void prepareAccessorTest() throws Exception {
    +        inputData = Mockito.mock(InputData.class);
    +
    +        Mockito.when(inputData.getDataSource()).thenReturn("TableTest");
    +
    +        columns.add(new ColumnDescriptor("id", DataType.INTEGER.getOID(), 0, "int4", null));
    +        columns.add(new ColumnDescriptor("name", DataType.TEXT.getOID(), 1, "text", null));
    +        columns.add(new ColumnDescriptor("birthday", DataType.DATE.getOID(), 2, "date", null));
    +        columns.add(new ColumnDescriptor("key", DataType.BYTEA.getOID(), 3, "bytea", null));
    +        Mockito.when(inputData.getTupleDescription()).thenReturn(columns);
    +        Mockito.when(inputData.getColumn(0)).thenReturn(columns.get(0));
    +        Mockito.when(inputData.getColumn(1)).thenReturn(columns.get(1));
    +        Mockito.when(inputData.getColumn(2)).thenReturn(columns.get(2));
    +        Mockito.when(inputData.getColumn(3)).thenReturn(columns.get(3));
    +    }
    +
    +    @Test
    +    public void testReadAccess() throws Exception {
    +        IgniteAccessor acc = PowerMockito.spy(new IgniteAccessor(inputData));
    +
    +        JsonObject correctAnswer = new JsonObject();
    +        JsonArray tempArray = new JsonArray();
    +        JsonArray tempArray2 = new JsonArray();
    +        tempArray2.add(1);
    +        tempArray2.add("abcd");
    +        tempArray2.add("'2001-01-01'");
    +        tempArray2.add("YWJjZA==");
    +        tempArray.add(tempArray2);
    +        correctAnswer.add("items", tempArray);
    +        correctAnswer.addProperty("last", false);
    +        correctAnswer.addProperty("queryId", 1);
    +
    +        ArgumentCaptor<String> captor = ArgumentCaptor.forClass(String.class);
    +        PowerMockito.doReturn(correctAnswer).when(acc, "sendRestRequest", anyString());
    +
    +        acc.openForRead();
    +        acc.readNextObject();
    +        acc.closeForRead();
    +
    +        PowerMockito.verifyPrivate(acc, Mockito.times(3)).invoke(method(IgniteAccessor.class, "sendRestRequest", String.class)).withArguments(captor.capture());
    +
    +        List<String> allParams = captor.getAllValues();
    +
    +        assertEquals(allParams.get(0), "http://127.0.0.1:8080/ignite?cmd=qryfldexe&pageSize=0&qry=SELECT+id%2Cname%2Cbirthday%2Ckey+FROM+TableTest");
    --- End diff --
    
    just wondering - when do the spaces in the query get replaced by `+`?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171072088
  
    --- Diff: pxf/pxf-ignite/README.md ---
    @@ -0,0 +1,67 @@
    +# Accessing Ignite database using PXF
    +
    +The PXF Ignite plug-in enables to access the [Ignite database](https://ignite.apache.org/) (both read and write operations are supported) via REST API.
    +
    +
    +# Prerequisites
    +
    +Check the following before using the plug-in:
    +
    +* The Ignite plug-in is installed on all PXF nodes;
    +
    +* The Ignite client is installed and running at the `IGNITE_HOST` (`localhost` by default; see below), and it accepts http queries from the PXF (note that *enabling Ignite REST API does not require changes in Ignite configuration*; see the instruction on how to do that at https://apacheignite.readme.io/docs/rest-api#section-getting-started).
    +
    +
    +# Syntax
    +
    +```
    +CREATE [READABLE] EXTERNAL TABLE <table_name> (
    +    <column_name> <data_type>[, <column_name> <data_type>, ...] | LIKE <other_table>
    +)
    +LOCATION ('pxf://<ignite_table_name>?PROFILE=Ignite[&<extra-parameter>&<extra-parameter>&...]')
    +FORMAT 'CUSTOM' (formatter='pxfwritable_import');
    +```
    +where each `<extra-parameter>` is one of the following:
    +* `IGNITE_HOST=<ignite_host_address_with_port>`. The location of Ignite client node. If not given, `127.0.0.1:8080` is used by default;
    +* `IGNITE_CACHE=<ignite_cache_name>`. The name of Ignite cache to use. If not given, this parameter is not included in queries from PXF to Ignite, thus Ignite default values will be used (at the moment, this is `Default` cache). This option is **case-sensitive**;
    +* `BUFFER_SIZE=<unsigned_int>`. The number of tuples send to (from) Ignite per a response. The same number of tuples is stored in in-plug-in cache. The values `0` and `1` are equal (cache is not used, each tuple is passed in it's own query to Ignite). If not given, `128` is used by default;
    --- End diff --
    
    typo it's -> its


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171118770
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/WhereSQLBuilder.java ---
    @@ -0,0 +1,140 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.LogicalFilter;
    +import org.apache.hawq.pxf.api.BasicFilter;
    +import org.apache.hawq.pxf.api.FilterParser;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +
    +/**
    + * Parse filter object generated by parent class  {@link org.apache.hawq.pxf.plugins.ignite.IgniteFilterBuilder},
    + * and build WHERE statement.
    + * For Multiple filters , currently only support HDOP_AND .
    + * The unsupported Filter operation and  LogicalOperation ,will return null statement.
    + *
    + */
    +public class WhereSQLBuilder extends IgniteFilterBuilder {
    +    private InputData inputData;
    +
    +    public WhereSQLBuilder(InputData input) {
    +        inputData = input;
    +    }
    +
    +    /**
    +     * 1.check for LogicalOperator, Jdbc currently only support HDOP_AND.
    --- End diff --
    
    typo jdbc?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171133073
  
    --- Diff: pxf/pxf-ignite/src/test/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenterTest.java ---
    @@ -0,0 +1,268 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import java.util.Calendar;
    +import java.util.List;
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.ByteUtil;
    +
    +import org.junit.Before;
    +import org.junit.Test;
    +import static org.junit.Assert.*;
    +
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +public class IgnitePartitionFragmenterTest {
    +    InputData inputData;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        prepareConstruction();
    +        when(inputData.getDataSource()).thenReturn("sales");
    +    }
    +
    +    @Test
    +    public void testPartionByDateOfMonth() throws Exception {
    +        when(inputData.getUserProperty("PARTITION_BY")).thenReturn("cdate:date");
    +        when(inputData.getUserProperty("RANGE")).thenReturn("2008-01-01:2009-01-01");
    +        when(inputData.getUserProperty("INTERVAL")).thenReturn("1:month");
    +
    +        IgnitePartitionFragmenter fragment = new IgnitePartitionFragmenter(inputData);
    +        List<Fragment> fragments = fragment.getFragments();
    +        assertEquals(12, fragments.size());
    +
    +        //fragment - 1
    +        byte[] fragMeta = fragments.get(0).getMetadata();
    +        byte[][] newBytes = ByteUtil.splitBytes(fragMeta, 8);
    +        long fragStart = ByteUtil.toLong(newBytes[0]);
    +        long fragEnd = ByteUtil.toLong(newBytes[1]);
    +        assertDateEquals(fragStart, 2008, 1, 1);
    +        assertDateEquals(fragEnd, 2008, 2, 1);
    +
    +        //fragment - 12
    +        fragMeta = fragments.get(11).getMetadata();
    +        newBytes = ByteUtil.splitBytes(fragMeta, 8);
    +        fragStart = ByteUtil.toLong(newBytes[0]);
    +        fragEnd = ByteUtil.toLong(newBytes[1]);
    +        assertDateEquals(fragStart, 2008, 12, 1);
    +        assertDateEquals(fragEnd, 2009, 1, 1);
    +
    +        //when end_date > start_date
    +        when(inputData.getUserProperty("RANGE")).thenReturn("2008-01-01:2001-01-01");
    +        fragment = new IgnitePartitionFragmenter(inputData);
    +        fragments = fragment.getFragments();
    +        assertEquals(0, fragments.size());
    +    }
    +
    +    @Test
    +    public void testPartionByDateOfYear() throws Exception {
    +        when(inputData.getUserProperty("PARTITION_BY")).thenReturn("cdate:date");
    +        when(inputData.getUserProperty("RANGE")).thenReturn("2008-01-01:2011-01-01");
    +        when(inputData.getUserProperty("INTERVAL")).thenReturn("1:year");
    +
    +        IgnitePartitionFragmenter fragment = new IgnitePartitionFragmenter(inputData);
    +        List<Fragment> fragments = fragment.getFragments();
    +        assertEquals(3, fragments.size());
    +    }
    +
    +    @Test
    +    public void testPartionByInt() throws Exception {
    +        when(inputData.getUserProperty("PARTITION_BY")).thenReturn("year:int");
    +        when(inputData.getUserProperty("RANGE")).thenReturn("2001:2012");
    +        when(inputData.getUserProperty("INTERVAL")).thenReturn("2");
    +
    +        IgnitePartitionFragmenter fragment = new IgnitePartitionFragmenter(inputData);
    +        List<Fragment> fragments = fragment.getFragments();
    +        assertEquals(6, fragments.size());
    +
    +        //fragment - 1
    +        byte[] fragMeta = fragments.get(0).getMetadata();
    +        byte[][] newBytes = ByteUtil.splitBytes(fragMeta, 4);
    +        int fragStart = ByteUtil.toInt(newBytes[0]);
    +        int fragEnd = ByteUtil.toInt(newBytes[1]);
    +        assertEquals(2001, fragStart);
    +        assertEquals(2003, fragEnd);
    +
    +        //fragment - 6
    +        fragMeta = fragments.get(5).getMetadata();
    +        newBytes = ByteUtil.splitBytes(fragMeta, 4);
    +        fragStart = ByteUtil.toInt(newBytes[0]);
    +        fragEnd = ByteUtil.toInt(newBytes[1]);
    +        assertEquals(2011, fragStart);
    +        assertEquals(2012, fragEnd);
    +
    +        //when end > start
    +        when(inputData.getUserProperty("RANGE")).thenReturn("2013:2012");
    +        fragment = new IgnitePartitionFragmenter(inputData);
    +        assertEquals(0, fragment.getFragments().size());
    +    }
    +
    +    @Test
    +    public void testPartionByEnum() throws Exception {
    +        when(inputData.getUserProperty("PARTITION_BY")).thenReturn("level:enum");
    +        when(inputData.getUserProperty("RANGE")).thenReturn("excellent:good:general:bad");
    +
    +        IgnitePartitionFragmenter fragment = new IgnitePartitionFragmenter(inputData);
    +        List<Fragment> fragments = fragment.getFragments();
    +        assertEquals(4, fragments.size());
    +
    +        //fragment - 1
    +        byte[] fragMeta = fragments.get(0).getMetadata();
    +        assertEquals("excellent", new String(fragMeta));
    +
    +        //fragment - 4
    +        fragMeta = fragments.get(3).getMetadata();
    +        assertEquals("bad", new String(fragMeta));
    +    }
    +
    +    @Test(expected = UserDataException.class)
    +    public void testInValidPartitiontype() throws Exception {
    +        when(inputData.getUserProperty("PARTITION_BY")).thenReturn("level:float");
    +        when(inputData.getUserProperty("RANGE")).thenReturn("100:200");
    +
    +        new IgnitePartitionFragmenter(inputData);
    +    }
    +
    +    @Test(expected = UserDataException.class)
    +    public void testInValidParameterFormat() throws Exception {
    +        //PARTITION_BY must be comma-delimited string
    +        when(inputData.getUserProperty("PARTITION_BY")).thenReturn("level-enum");
    +        when(inputData.getUserProperty("RANGE")).thenReturn("100:200");
    +
    +        new IgnitePartitionFragmenter(inputData);
    +    }
    +
    +    @Test(expected = UserDataException.class)
    +    public void testInValidDateFormat() throws Exception {
    +        //date string must be yyyy-MM-dd
    +        when(inputData.getUserProperty("PARTITION_BY")).thenReturn("cdate:date");
    +        when(inputData.getUserProperty("RANGE")).thenReturn("2008/01/01:2009-01-01");
    +        when(inputData.getUserProperty("INTERVAL")).thenReturn("1:month");
    +
    +        new IgnitePartitionFragmenter(inputData).getFragments();
    +    }
    +
    +    @Test(expected = UserDataException.class)
    +    public void testInValidParameterValue() throws Exception {
    +        //INTERVAL must be greater than 0
    +        when(inputData.getUserProperty("PARTITION_BY")).thenReturn("cdate:date");
    +        when(inputData.getUserProperty("RANGE")).thenReturn("2008-01-01:2009-01-01");
    +        when(inputData.getUserProperty("INTERVAL")).thenReturn("-1:month");
    +
    +        new IgnitePartitionFragmenter(inputData);
    +    }
    +
    +    @Test(expected = UserDataException.class)
    +    public void testInValidIntervaltype() throws Exception {
    --- End diff --
    
    typo IntervalType (and in below tests)


---

[GitHub] incubator-hawq issue #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344
  
    @denalex @hornn @sansanichfb Is there anything we need to do/fix for this PR to be merged?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171116788
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteResolver.java ---
    @@ -0,0 +1,173 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +
    +import java.util.ArrayList;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.text.SimpleDateFormat;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.codec.binary.Base64;
    +import org.apache.commons.codec.binary.Hex;
    +
    +import com.google.gson.JsonArray;
    +
    +
    --- End diff --
    
    remove extra new lines?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171055669
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    +        return true;
    +    }
    +
    +    /**
    +     * readNextObject() implementation
    +     */
    +    @Override
    +    public OneRow readNextObject() throws Exception {
    +        if (urlReadFetch == null) {
    +            LOG.error("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +            throw new ProtocolException("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +        }
    +
    +        if (bufferRead.isEmpty()) {
    +            // Refill buffer
    +            if (isLastReadFinished) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +
    +            JsonElement response = sendRestRequest(urlReadFetch);
    +            isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +
    +            // Parse 'items'
    +            Iterator<JsonElement> itemsIterator = response.getAsJsonObject().get("items").getAsJsonArray().iterator();
    +            while (itemsIterator.hasNext()) {
    +                if (!bufferRead.add(itemsIterator.next().getAsJsonArray())) {
    +                    LOG.error("readNextObject(): Buffer refill failed (not enough memory in 'bufferRead')");
    +                    throw new OutOfMemoryError("readNextObject(): not enough memory in 'bufferRead'");
    +                }
    +            }
    +
    +            // Check again in case "response" contains no elements
    +            if (bufferRead.isEmpty()) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): Buffer refill failed");
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +        }
    +
    +        return new OneRow(bufferRead.pollFirst());
    +    }
    +
    +    /**
    +     * closeForRead() implementation
    +     */
    +    @Override
    +    public void closeForRead() {
    +        if (urlReadClose != null) {
    +            try {
    +                sendRestRequest(urlReadClose);
    +            }
    +            catch (Exception e) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("closeForRead() Exception: " + e.getClass().getSimpleName());
    +                }
    +            }
    +        }
    +        isLastReadFinished = false;
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Ignite read request finished. URL: '" + urlReadClose + "'");
    +        }
    +    }
    +
    +    /**
    +     * openForWrite() implementation.
    +     * No queries are sent to Ignite by this procedure, so if there are some problems (for example, with connection), they will be revealed only during the execution of 'writeNextObject()'
    +     */
    +    @Override
    +    public boolean openForWrite() throws UserDataException {
    +        // This is a temporary solution. At the moment there is no other way (except for the usage of user-defined parameters) to get the correct name of Ignite table: GPDB inserts extra data into the address, as required by Hadoop.
    +        // Note that if no extra data is present, the 'definedSource' will be left unchanged
    +        String definedSource = inputData.getDataSource();
    +        Pattern pattern = Pattern.compile("/(.*)/[0-9]*-[0-9]*_[0-9]*");
    +        Matcher matcher = pattern.matcher(definedSource);
    +        if (matcher.find()) {
    +            inputData.setDataSource(matcher.group(1));
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("INSERT INTO ");
    +
    +        // Insert the table name
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert the column names
    +        sb.append("(");
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        String fieldDivisor = "";
    +        for (int i = 0; i < columns.size(); i++) {
    +            sb.append(fieldDivisor);
    +            fieldDivisor = ", ";
    +            sb.append(columns.get(i).columnName());
    +        }
    +        sb.append(")");
    +
    +        sb.append(" VALUES ");
    +
    +        queryWrite = sb.toString();
    +        return true;
    +    }
    +
    +    /**
    +     * writeNextObject() implementation
    +     */
    +    @Override
    +    public boolean writeNextObject(OneRow currentRow) throws Exception {
    +        boolean currentRowInBuffer = bufferWrite.add(currentRow);
    +
    +        if (!isWriteActive) {
    +            if (!currentRowInBuffer) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferWrite')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferWrite'");
    +            }
    +            LOG.info("Ignite write request. Query: '" + queryWrite + "'");
    +            sendInsertRestRequest(queryWrite);
    +            bufferWrite.removeFirst();
    +            isWriteActive = true;
    +            return true;
    +        }
    +
    +        if ((bufferWrite.size() >= bufferSize) || (!currentRowInBuffer)) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +
    +        if (!currentRowInBuffer) {
    +            if (!bufferWrite.add(currentRow)) {
    +                LOG.error("writeNextObject(): Failed (not enough memory in 'bufferSend')");
    +                throw new OutOfMemoryError("writeNextObject(): not enough memory in 'bufferSend'");
    +            }
    +        }
    +        
    +        return true;
    +    }
    +
    +    /**
    +     * closeForWrite() implementation
    +     */
    +    @Override
    +    public void closeForWrite() throws Exception {
    +        if (!bufferWrite.isEmpty()) {
    +            sendInsertRestRequest(queryWrite);
    +        }
    +        if (isWriteActive) {
    +            // At this point, the request must have finished successfully; otherwise an exception would be thrown before
    +            LOG.info("Ignite write request finished successfully. Query: '" + queryWrite + "'");
    +        }
    +        isWriteActive = false;
    +    }
    +
    +    /**
    +     * Build HTTP GET query for Ignite REST API with command 'qryfldexe'
    +     * 
    +     * @param querySql SQL query with filter constants. The constants must replaced by "?" if 'filterConstants' is not given
    +     * @param filterConstants
    +     * 
    +     * @return Prepared HTTP query. The query will be properly encoded with {@link java.net.URLEncoder}
    +     * 
    +     * @throws UnsupportedEncodingException from {@link java.net.URLEncoder.encode()}
    +     */
    +    private String buildQueryFldexe(String querySql, ArrayList<String> filterConstants) throws UnsupportedEncodingException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("http://");
    +        sb.append(igniteHost);
    +        sb.append("/ignite");
    +        sb.append("?");
    +        sb.append("cmd=qryfldexe");
    +        sb.append("&");
    +        sb.append("pageSize=0");
    +        sb.append("&");
    +        if (cacheName != null) {
    +            sb.append("cacheName=");
    +            // Note that Ignite supports only "good" cache names (those that should be left unchanged by the URLEncoder.encode())
    +            sb.append(URLEncoder.encode(cacheName, "UTF-8"));
    +            sb.append("&");
    +        }
    +        int counter = 1;
    +        if (filterConstants != null) {
    +            for (String constant : filterConstants) {
    +                sb.append("arg");
    +                sb.append(counter);
    +                sb.append("=");
    +                sb.append(URLEncoder.encode(constant, "UTF-8"));
    +                sb.append("&");
    +                counter += 1;
    +            }
    +        }
    +        sb.append("qry=");
    +        sb.append(URLEncoder.encode(querySql, "UTF-8"));
    +
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Build HTTP GET query for Ignite REST API with command 'qryfetch'
    +     * This query is used to retrieve data after the 'qryfldexe' command started
    +     * 
    +     * @param queryId ID of the query assigned by Ignite when the query started
    +     *
    +     * @return Prepared HTTP query
    +     */
    +    private String buildQueryFetch(int queryId) {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("http://");
    +        sb.append(igniteHost);
    +        sb.append("/ignite");
    +        sb.append("?");
    +        sb.append("cmd=qryfetch");
    +        sb.append("&");
    +        sb.append("pageSize=");
    +        sb.append(bufferSize);
    +        sb.append("&");
    +        sb.append("qryId=");
    +        sb.append(queryId);
    +
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Build HTTP GET query for Ignite REST API with command 'qrycls'
    +     * This query is used to close query resources on Ignite side
    +     * 
    +     * @param queryId ID of the query assigned by Ignite when the query started
    +     *
    +     * @return Prepared HTTP query
    +     */
    +    private String buildQueryCls(int queryId) {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("http://");
    +        sb.append(igniteHost);
    +        sb.append("/ignite");
    +        sb.append("?");
    +        sb.append("cmd=qrycls");
    +        sb.append("&");
    +        sb.append("qryId=");
    +        sb.append(queryId);
    +
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Send a REST request to the Ignite server
    +     * 
    +     * @param query A prepared and properly encoded HTTP GET request
    +     * 
    +     * @return "response" field from the received JSON object
    +     * (See Ignite REST API documentation for details)
    +     * 
    +     * @throws ProtocolException if Ignite reports error in it's JSON response
    +     * @throws MalformedURLException if URL is malformed
    +     * @throws IOException in case of connection failure
    +     */
    +    private JsonElement sendRestRequest(String query) throws ProtocolException, MalformedURLException, IOException {
    +        // Create URL object
    +        URL url;
    +        try {
    +            url = new URL(query);
    +        }
    +        catch (MalformedURLException e) {
    +            LOG.error("sendRestRequest(): Failed (malformed URL). URL is '" + query + "'");
    +            throw e;
    +        }
    +
    +        // Connect to the Ignite server, send query and get raw response
    +        String responseRaw = null;
    +        try {
    +            StringBuilder sb = new StringBuilder();
    +            BufferedReader reader = new BufferedReader(new InputStreamReader(url.openStream()));
    +            String responseLine;
    +            while ((responseLine = reader.readLine()) != null) {
    +                sb.append(responseLine);
    +            }
    +            reader.close();
    --- End diff --
    
    should this line go into finally block, in case reading fails and throws an Exception ?


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171117117
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteResolver.java ---
    @@ -0,0 +1,173 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +
    +import java.util.ArrayList;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.text.SimpleDateFormat;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.codec.binary.Base64;
    +import org.apache.commons.codec.binary.Hex;
    +
    +import com.google.gson.JsonArray;
    +
    +
    +
    +/**
    + * PXF-Ignite resolver class
    + */
    +public class IgniteResolver extends IgnitePlugin implements ReadResolver, WriteResolver {
    +    private static final Log LOG = LogFactory.getLog(IgniteResolver.class);
    +    
    +    // HAWQ column descriptors
    +    private ArrayList<ColumnDescriptor> columns = null;
    +
    +    public IgniteResolver(InputData input) throws Exception {
    +        super(input);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        columns = input.getTupleDescription();
    +        
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful");
    +        }
    +    }
    +
    +    /**
    +     * Transform a JsonArray object stored in {@link OneRow} into a List {@link OneField}
    +     */
    +    @Override
    +    public List<OneField> getFields(OneRow row) throws Exception {
    +        JsonArray result = (JsonArray)row.getData();
    --- End diff --
    
    just wondering - does this line generate a warning? if so, we should make a safer cast or suppress the warning.


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171131203
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenter.java ---
    @@ -0,0 +1,384 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.FragmentsStats;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +
    +import java.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.commons.lang.ArrayUtils;
    +
    +/**
    + * Fragmenter class for Ignite data resources.
    + * @note At the moment this fragmenter works just like the JDBC's one
    + *
    + * Aside from implementing PXF {@link Fragmenter},
    + * this class also transforms input data path (given by user) by adding the
    + * information on splitting the request into a few requests for different regions.
    + * <br>
    + * The parameter Patterns
    + * <br>
    + * There are three  parameters, the format is as follows:
    + * <br>
    + * <pre>
    + * <code>PARTITION_BY=column_name:column_type&amp;RANGE=start_value[:end_value]&amp;INTERVAL=interval_num[:interval_unit]</code>
    + * </pre>
    + * The <code>PARTITION_BY</code> parameter can be split by colon(':'),the <code>column_type</code> current supported : <code>date,int,enum</code> .
    + * The Date format is 'yyyy-MM-dd'. <br>
    + * The <code>RANGE</code> parameter can be split by colon(':') ,used to identify the starting range of each fragment.
    + * The range is left-closed, ie:<code> '&gt;= start_value AND &lt; end_value' </code>.If the <code>column_type</code> is <code>int</code>,
    + * the <code>end_value</code> can be empty. If the <code>column_type</code>is <code>enum</code>,the parameter <code>RANGE</code> can be empty. <br>
    + * The <code>INTERVAL</code> parameter can be split by colon(':'), indicate the interval value of one fragment.
    + * When <code>column_type</code> is <code>date</code>,this parameter must be split by colon, and <code>interval_unit</code> can be <code>year,month,day</code>.
    + * When <code>column_type</code> is <code>int</code>, the <code>interval_unit</code> can be empty.
    + * When <code>column_type</code> is <code>enum</code>,the <code>INTERVAL</code> parameter can be empty.
    + * <br>
    + * <p>
    + * The syntax examples is:
    + * <br>
    + * <code>PARTITION_BY=createdate:date&amp;RANGE=2008-01-01:2010-01-01&amp;INTERVAL=1:month'</code>
    + * <br>
    + * <code>PARTITION_BY=year:int&amp;RANGE=2008:2010&amp;INTERVAL=1</code>
    + * <br>
    + * <code>PARTITION_BY=grade:enum&amp;RANGE=excellent:good:general:bad</code>
    + * </p>
    + */
    +public class IgnitePartitionFragmenter extends Fragmenter {
    +    Log LOG = LogFactory.getLog(IgnitePartitionFragmenter.class);
    +
    +    String[] partitionBy = null;
    +    String[] range = null;
    +    String[] interval = null;
    +    PartitionType partitionType = null;
    +    String partitionColumn = null;
    +    IntervalType intervalType = null;
    +    int intervalNum = 1;
    +
    +    //when partitionType is DATE,it is valid
    +    Calendar rangeStart = null;
    +    Calendar rangeEnd = null;
    +
    +
    +    enum PartitionType {
    +        DATE,
    +        INT,
    +        ENUM;
    +
    +        public static PartitionType getType(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    enum IntervalType {
    +        DAY,
    +        MONTH,
    +        YEAR;
    +
    +        public static IntervalType type(String str) {
    +            return valueOf(str.toUpperCase());
    +        }
    +    }
    +
    +    /**
    +     * @throws UserDataException if the request parameter is malformed
    +     */
    +    public IgnitePartitionFragmenter(InputData inConf) throws UserDataException {
    +        super(inConf);
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor started");
    +        }
    +
    +        if (inConf.getUserProperty("PARTITION_BY") == null) {
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("Constructor successful; partition was not used");
    +            }
    +            return;
    +        }
    +
    +        try {
    +            partitionBy = inConf.getUserProperty("PARTITION_BY").split(":");
    +            partitionColumn = partitionBy[0];
    +            partitionType = PartitionType.getType(partitionBy[1]);
    +        }
    +        catch (IllegalArgumentException | ArrayIndexOutOfBoundsException e1) {
    +            throw new UserDataException("The parameter 'PARTITION_BY' invalid, the pattern is 'column_name:date|int|enum'");
    +        }
    +
    +        //parse and validate parameter-RANGE
    +        try {
    +            String rangeStr = inConf.getUserProperty("RANGE");
    +            if (rangeStr != null) {
    +                range = rangeStr.split(":");
    +                if (range.length == 1 && partitionType != PartitionType.ENUM) {
    +                    throw new UserDataException("The parameter 'RANGE' does not specify '[:end_value]'");
    +                }
    +            } 
    +            else {
    +                throw new UserDataException("The parameter 'RANGE' must be specified along with 'PARTITION_BY'");
    +            }
    +        }
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'RANGE' invalid, the pattern is 'start_value[:end_value]'");
    +        }
    +
    +        //parse and validate parameter-INTERVAL
    +        try {
    +            String intervalStr = inConf.getUserProperty("INTERVAL");
    +            if (intervalStr != null) {
    +                interval = intervalStr.split(":");
    +                intervalNum = Integer.parseInt(interval[0]);
    +                if (interval.length > 1) {
    +                    intervalType = IntervalType.type(interval[1]);
    +                }
    +                if (interval.length == 1 && partitionType == PartitionType.DATE) {
    +                    throw new UserDataException("The parameter 'INTERVAL' does not specify unit [:year|month|day]");
    +                }
    +            } 
    +            else if (partitionType != PartitionType.ENUM) {
    +                throw new UserDataException("The parameter 'INTERVAL' must be specified along with 'PARTITION_BY'");
    +            }
    +            if (intervalNum < 1) {
    +                throw new UserDataException("The parameter 'INTERVAL' must > 1, but actual is '" + intervalNum + "'");
    +            }
    +        } 
    +        catch (IllegalArgumentException e) {
    +            throw new UserDataException("The parameter 'INTERVAL' invalid, the pattern is 'interval_num[:interval_unit]'");
    +        }
    +
    +        //parse any date values
    +        try {
    +            if (partitionType == PartitionType.DATE) {
    +                SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd");
    +                rangeStart = Calendar.getInstance();
    +                rangeStart.setTime(df.parse(range[0]));
    +                rangeEnd = Calendar.getInstance();
    +                rangeEnd.setTime(df.parse(range[1]));
    +            }
    +        } catch (ParseException e) {
    +            throw new UserDataException("The parameter 'RANGE' has invalid date format. Expected format is 'YYYY-MM-DD'");
    +        }
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Constructor successful; some partition used");
    +        }
    +    }
    +
    +    /**
    +     * Returns statistics for  Ignite table. Currently it's not implemented.
    --- End diff --
    
    typo remove space before Ignite


---

[GitHub] incubator-hawq pull request #1344: HAWQ-1599. PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171133652
  
    --- Diff: pxf/pxf-ignite/src/test/java/org/apache/hawq/pxf/plugins/ignite/IgnitePartitionFragmenterTest.java ---
    @@ -0,0 +1,268 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import java.util.Calendar;
    +import java.util.List;
    +
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.ByteUtil;
    +
    +import org.junit.Before;
    +import org.junit.Test;
    +import static org.junit.Assert.*;
    +
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +public class IgnitePartitionFragmenterTest {
    +    InputData inputData;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        prepareConstruction();
    +        when(inputData.getDataSource()).thenReturn("sales");
    +    }
    +
    +    @Test
    +    public void testPartionByDateOfMonth() throws Exception {
    +        when(inputData.getUserProperty("PARTITION_BY")).thenReturn("cdate:date");
    +        when(inputData.getUserProperty("RANGE")).thenReturn("2008-01-01:2009-01-01");
    +        when(inputData.getUserProperty("INTERVAL")).thenReturn("1:month");
    +
    +        IgnitePartitionFragmenter fragment = new IgnitePartitionFragmenter(inputData);
    +        List<Fragment> fragments = fragment.getFragments();
    +        assertEquals(12, fragments.size());
    +
    +        //fragment - 1
    +        byte[] fragMeta = fragments.get(0).getMetadata();
    +        byte[][] newBytes = ByteUtil.splitBytes(fragMeta, 8);
    +        long fragStart = ByteUtil.toLong(newBytes[0]);
    +        long fragEnd = ByteUtil.toLong(newBytes[1]);
    +        assertDateEquals(fragStart, 2008, 1, 1);
    +        assertDateEquals(fragEnd, 2008, 2, 1);
    +
    +        //fragment - 12
    +        fragMeta = fragments.get(11).getMetadata();
    +        newBytes = ByteUtil.splitBytes(fragMeta, 8);
    +        fragStart = ByteUtil.toLong(newBytes[0]);
    +        fragEnd = ByteUtil.toLong(newBytes[1]);
    +        assertDateEquals(fragStart, 2008, 12, 1);
    +        assertDateEquals(fragEnd, 2009, 1, 1);
    +
    +        //when end_date > start_date
    +        when(inputData.getUserProperty("RANGE")).thenReturn("2008-01-01:2001-01-01");
    +        fragment = new IgnitePartitionFragmenter(inputData);
    +        fragments = fragment.getFragments();
    +        assertEquals(0, fragments.size());
    +    }
    +
    +    @Test
    +    public void testPartionByDateOfYear() throws Exception {
    +        when(inputData.getUserProperty("PARTITION_BY")).thenReturn("cdate:date");
    +        when(inputData.getUserProperty("RANGE")).thenReturn("2008-01-01:2011-01-01");
    +        when(inputData.getUserProperty("INTERVAL")).thenReturn("1:year");
    +
    +        IgnitePartitionFragmenter fragment = new IgnitePartitionFragmenter(inputData);
    +        List<Fragment> fragments = fragment.getFragments();
    +        assertEquals(3, fragments.size());
    +    }
    +
    +    @Test
    +    public void testPartionByInt() throws Exception {
    +        when(inputData.getUserProperty("PARTITION_BY")).thenReturn("year:int");
    +        when(inputData.getUserProperty("RANGE")).thenReturn("2001:2012");
    +        when(inputData.getUserProperty("INTERVAL")).thenReturn("2");
    +
    +        IgnitePartitionFragmenter fragment = new IgnitePartitionFragmenter(inputData);
    +        List<Fragment> fragments = fragment.getFragments();
    +        assertEquals(6, fragments.size());
    +
    +        //fragment - 1
    +        byte[] fragMeta = fragments.get(0).getMetadata();
    +        byte[][] newBytes = ByteUtil.splitBytes(fragMeta, 4);
    +        int fragStart = ByteUtil.toInt(newBytes[0]);
    +        int fragEnd = ByteUtil.toInt(newBytes[1]);
    +        assertEquals(2001, fragStart);
    +        assertEquals(2003, fragEnd);
    +
    +        //fragment - 6
    +        fragMeta = fragments.get(5).getMetadata();
    +        newBytes = ByteUtil.splitBytes(fragMeta, 4);
    +        fragStart = ByteUtil.toInt(newBytes[0]);
    +        fragEnd = ByteUtil.toInt(newBytes[1]);
    +        assertEquals(2011, fragStart);
    +        assertEquals(2012, fragEnd);
    +
    +        //when end > start
    +        when(inputData.getUserProperty("RANGE")).thenReturn("2013:2012");
    +        fragment = new IgnitePartitionFragmenter(inputData);
    +        assertEquals(0, fragment.getFragments().size());
    +    }
    +
    +    @Test
    +    public void testPartionByEnum() throws Exception {
    +        when(inputData.getUserProperty("PARTITION_BY")).thenReturn("level:enum");
    +        when(inputData.getUserProperty("RANGE")).thenReturn("excellent:good:general:bad");
    +
    +        IgnitePartitionFragmenter fragment = new IgnitePartitionFragmenter(inputData);
    +        List<Fragment> fragments = fragment.getFragments();
    +        assertEquals(4, fragments.size());
    +
    +        //fragment - 1
    +        byte[] fragMeta = fragments.get(0).getMetadata();
    +        assertEquals("excellent", new String(fragMeta));
    +
    +        //fragment - 4
    +        fragMeta = fragments.get(3).getMetadata();
    +        assertEquals("bad", new String(fragMeta));
    +    }
    +
    +    @Test(expected = UserDataException.class)
    +    public void testInValidPartitiontype() throws Exception {
    --- End diff --
    
    typo Invalid (and in tests below)


---

[GitHub] incubator-hawq pull request #1344: PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344#discussion_r171045985
  
    --- Diff: pxf/pxf-ignite/src/main/java/org/apache/hawq/pxf/plugins/ignite/IgniteAccessor.java ---
    @@ -0,0 +1,502 @@
    +package org.apache.hawq.pxf.plugins.ignite;
    +
    +/*
    + * 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.
    + */
    +
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.UserDataException;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.ignite.IgnitePlugin;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.io.InputStreamReader;
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.UnsupportedEncodingException;
    +import java.net.URL;
    +import java.net.URLEncoder;
    +import java.net.MalformedURLException;
    +import java.net.ProtocolException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +
    +import com.google.gson.JsonParser;
    +import com.google.gson.JsonElement;
    +import com.google.gson.JsonArray;
    +
    +
    +/**
    + * Ignite database read and write accessor
    + */
    +public class IgniteAccessor extends IgnitePlugin implements ReadAccessor, WriteAccessor {
    +    private static final Log LOG = LogFactory.getLog(IgniteAccessor.class);
    +
    +    // Prepared URLs to send to Ignite when reading data
    +    private String urlReadStart = null;
    +    private String urlReadFetch = null;
    +    private String urlReadClose = null;
    +    // Set to true when Ignite reported all the data for the SELECT query was retreived
    +    private boolean isLastReadFinished = false;
    +    // A buffer to store the SELECT query results (without Ignite metadata)
    +    private LinkedList<JsonArray> bufferRead = new LinkedList<JsonArray>();
    +
    +    // A template for the INSERT 
    +    private String queryWrite = null;
    +    // Set to true when the INSERT operation is in progress
    +    private boolean isWriteActive = false;
    +    // A buffer to store prepared values for the INSERT query
    +    private LinkedList<OneRow> bufferWrite = new LinkedList<OneRow>();
    +    
    +
    +    /**
    +     * Class constructor.
    +     */
    +    public IgniteAccessor(InputData inputData) throws UserDataException {
    +        super(inputData);
    +    }
    +
    +    /**
    +     * openForRead() implementation
    +     */
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        if (bufferSize == 0) {
    +            bufferSize = 1;
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        // Insert a list of fields to be selected
    +        ArrayList<ColumnDescriptor> columns = inputData.getTupleDescription();
    +        if (columns == null) {
    +            throw new UserDataException("Tuple description must be present.");
    +        }
    +        sb.append("SELECT ");
    +        for (int i = 0; i < columns.size(); i++) {
    +            ColumnDescriptor column = columns.get(i);
    +            if (i > 0) {
    +                sb.append(",");
    +            }
    +            sb.append(column.columnName());
    +        }
    +
    +        // Insert the name of the table to select values from
    +        sb.append(" FROM ");
    +        String tableName = inputData.getDataSource();
    +        if (tableName == null) {
    +            throw new UserDataException("Table name must be set as DataSource.");
    +        }
    +        sb.append(tableName);
    +
    +        // Insert query constraints
    +        // Note: Filter constants may be provided separately from the query itself, mostly for the safety of the SQL queries; at the moment, however, they are passed in the query itself.
    +        ArrayList<String> filterConstants = null;
    +        if (inputData.hasFilter()) {
    +            WhereSQLBuilder filterBuilder = new WhereSQLBuilder(inputData);
    +            String whereSql = filterBuilder.buildWhereSQL();
    +
    +            if (whereSql != null) {
    +                sb.append(" WHERE ").append(whereSql);
    +            }
    +        }
    +
    +        // Insert partition constaints
    +        sb = new IgnitePartitionFragmenter(inputData).buildFragmenterSql(sb);
    +
    +        // Format URL
    +        urlReadStart = buildQueryFldexe(sb.toString(), filterConstants);
    +
    +        // Send the first REST request that opens the connection
    +        JsonElement response = sendRestRequest(urlReadStart);
    +
    +        // Build 'urlReadFetch' and 'urlReadClose'
    +        isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +        urlReadFetch = buildQueryFetch(response.getAsJsonObject().get("queryId").getAsInt());
    +        urlReadClose = buildQueryCls(response.getAsJsonObject().get("queryId").getAsInt());
    +
    +        LOG.info("Ignite read request. URL: '" + urlReadStart + "'");
    +        return true;
    +    }
    +
    +    /**
    +     * readNextObject() implementation
    +     */
    +    @Override
    +    public OneRow readNextObject() throws Exception {
    +        if (urlReadFetch == null) {
    +            LOG.error("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +            throw new ProtocolException("readNextObject(): urlReadFetch is null. This means the Ignite qryfldexe query was not executed properly");
    +        }
    +
    +        if (bufferRead.isEmpty()) {
    +            // Refill buffer
    +            if (isLastReadFinished) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +
    +            JsonElement response = sendRestRequest(urlReadFetch);
    +            isLastReadFinished = response.getAsJsonObject().get("last").getAsBoolean();
    +
    +            // Parse 'items'
    +            Iterator<JsonElement> itemsIterator = response.getAsJsonObject().get("items").getAsJsonArray().iterator();
    +            while (itemsIterator.hasNext()) {
    +                if (!bufferRead.add(itemsIterator.next().getAsJsonArray())) {
    +                    LOG.error("readNextObject(): Buffer refill failed (not enough memory in 'bufferRead')");
    +                    throw new OutOfMemoryError("readNextObject(): not enough memory in 'bufferRead'");
    +                }
    +            }
    +
    +            // Check again in case "response" contains no elements
    +            if (bufferRead.isEmpty()) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("readNextObject(): Buffer refill failed");
    +                    LOG.debug("readNextObject(): All the data received from Ignite");
    +                }
    +                return null;
    +            }
    +        }
    +
    +        return new OneRow(bufferRead.pollFirst());
    +    }
    +
    +    /**
    +     * closeForRead() implementation
    +     */
    +    @Override
    +    public void closeForRead() {
    +        if (urlReadClose != null) {
    +            try {
    +                sendRestRequest(urlReadClose);
    +            }
    +            catch (Exception e) {
    +                if (LOG.isDebugEnabled()) {
    +                    LOG.debug("closeForRead() Exception: " + e.getClass().getSimpleName());
    +                }
    +            }
    +        }
    +        isLastReadFinished = false;
    +
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Ignite read request finished. URL: '" + urlReadClose + "'");
    +        }
    +    }
    +
    +    /**
    +     * openForWrite() implementation.
    +     * No queries are sent to Ignite by this procedure, so if there are some problems (for example, with connection), they will be revealed only during the execution of 'writeNextObject()'
    +     */
    +    @Override
    +    public boolean openForWrite() throws UserDataException {
    +        // This is a temporary solution. At the moment there is no other way (except for the usage of user-defined parameters) to get the correct name of Ignite table: GPDB inserts extra data into the address, as required by Hadoop.
    +        // Note that if no extra data is present, the 'definedSource' will be left unchanged
    +        String definedSource = inputData.getDataSource();
    +        Pattern pattern = Pattern.compile("/(.*)/[0-9]*-[0-9]*_[0-9]*");
    --- End diff --
    
    Pattern is thread-safe so can be used as a static final member not to be re-created for every write request.


---

[GitHub] incubator-hawq issue #1344: HAWQ-1599. PXF Ignite plugin

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

    https://github.com/apache/incubator-hawq/pull/1344
  
    Most of the proposed fixes were implemented, and I have also made some minor changes in the code.
    
    @denalex, @hornn, @sansanichfb could you please re-review the PR and make additional comments (if any)?
    
    As far as I understand, the Travis CI build fails due to some bug in the CI engine itself or its settings. Is that correct?
    
    @radarwave Thank you for your instruction! A JIRA task was created and the PR name changed accordingly.


---