You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@tajo.apache.org by hyunsik <gi...@git.apache.org> on 2015/07/29 07:59:17 UTC

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

GitHub user hyunsik opened a pull request:

    https://github.com/apache/tajo/pull/662

    TAJO-1699: Tajo Java Client version 2.

    

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

    $ git pull https://github.com/hyunsik/tajo CLIENT_v2

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

    https://github.com/apache/tajo/pull/662.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 #662
    
----
commit 4597561198cf1d55fcd39cb76fc142facb0e9d34
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-07-27T16:23:28Z

    Add initial design for client API v2.

commit 3223dc68433822e16a5f1624f4474b543afacb58
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-07-28T07:39:28Z

    Complete initial version for client API v2.

commit 1bea9f7052dd1650e8e9679c32113042a4a37dfb
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-07-28T07:53:15Z

    Add testExecuteUpdate test.

commit 79e2a5ba59123c1e76ae7b614ac9730937a40c86
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-07-28T16:00:43Z

    Add unit tests for 3 types SQL execution through client v2.

commit 54365db55dafc123d9aefab27da63b2bf2cba589
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-07-29T02:12:10Z

    Changed lots of SQLException to TajoException and implement ClientExceptionUtil.

commit 8c450111014c2ac44d719de963d31bef7d337d11
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-07-29T02:12:41Z

    Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/tajo into CLIENT_v2

commit 1c07bf984ad2c6b359b83dea3b5201f8b8a6418b
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-07-29T05:54:58Z

    Add listener and refactored exceptions's constructor to take ReturnState.

----


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36056346
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/v2/ClientDeligateFactory.java ---
    @@ -0,0 +1,42 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +import org.apache.tajo.annotation.Nullable;
    +import org.apache.tajo.client.v2.exception.ClientUnableToConnectException;
    +
    +import java.util.Map;
    +
    +public class ClientDeligateFactory {
    --- End diff --
    
    fixed


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35947774
  
    --- Diff: tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java ---
    @@ -91,7 +91,7 @@ public String getSystemDatabaseName() {
         return systemTableNames;
       }
       
    -  private TableDescriptor getTableDescriptor(String tableName) {
    +  private TableDescriptor getTableDescriptor(String tableName) throws UndefinedTableException {
    --- End diff --
    
    I know you just modified the function signature to throw an exception. However, I found a problem that this function returns null instead of throwing UndefinedTableException when the table of the given name is not found.
    Would you also check it please?


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35953155
  
    --- Diff: tajo-common/src/main/java/org/apache/tajo/exception/ExceptionUtil.java ---
    @@ -21,6 +21,11 @@
     import org.apache.commons.logging.Log;
     import org.apache.hadoop.util.StringUtils;
     import org.apache.tajo.common.TajoDataTypes.DataType;
    +import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35952534
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/v2/ClientDeligateFactory.java ---
    @@ -0,0 +1,42 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +import org.apache.tajo.annotation.Nullable;
    +import org.apache.tajo.client.v2.exception.ClientUnableToConnectException;
    +
    +import java.util.Map;
    +
    +public class ClientDeligateFactory {
    +
    +  public static ClientDelegate newDefaultDeligate(String host,
    --- End diff --
    
    Please fix typo.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36128726
  
    --- Diff: tajo-common/src/main/java/org/apache/tajo/exception/TajoException.java ---
    @@ -27,6 +28,16 @@
     public class TajoException extends Exception implements TajoExceptionInterface {
       private ResultCode code;
     
    +  public TajoException(ReturnState e) {
    --- End diff --
    
    Got it. 


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968835
  
    --- Diff: tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/HeapTuple.java ---
    @@ -23,6 +23,7 @@
     
     import org.apache.tajo.common.TajoDataTypes;
     import org.apache.tajo.datum.*;
    +import org.apache.tajo.exception.TajoRuntimeException;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36056057
  
    --- Diff: tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java ---
    @@ -179,7 +179,7 @@ public AbstractDBStore(Configuration conf) throws InternalException {
               }
             }
          }
    -    } catch (Exception se) {
    +    } catch (Throwable se) {
    --- End diff --
    
    Fixed.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on the pull request:

    https://github.com/apache/tajo/pull/662#issuecomment-125851824
  
    This work is highly related to the error propagation work. So, it also refactors error and exception related codes.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36056502
  
    --- Diff: tajo-core/src/test/java/org/apache/tajo/client/v2/TestTajoClientV2.java ---
    @@ -0,0 +1,236 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +import com.facebook.presto.hive.shaded.com.google.common.collect.Lists;
    +import org.apache.tajo.QueryTestCaseBase;
    +import org.apache.tajo.catalog.exception.DuplicateDatabaseException;
    +import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
    +import org.apache.tajo.catalog.exception.UndefinedTableException;
    +import org.apache.tajo.exception.TajoException;
    +import org.apache.tajo.service.ServiceTracker;
    +import org.apache.tajo.service.ServiceTrackerFactory;
    +import org.junit.AfterClass;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.io.IOException;
    +import java.net.InetSocketAddress;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.util.List;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import static org.junit.Assert.*;
    +
    +public class TestTajoClientV2 extends QueryTestCaseBase {
    +  private static TajoClient clientv2;
    +
    +  @BeforeClass
    +  public static void setUp() throws Exception {
    +    conf = testingCluster.getConfiguration();
    +
    +    clientv2 = new TajoClient(new ServiceDiscovery() {
    +      ServiceTracker tracker = ServiceTrackerFactory.get(conf);
    +      @Override
    +      public InetSocketAddress clientAddress() {
    +        return tracker.getClientServiceAddress();
    +      }
    +    });
    +  }
    +
    +  @AfterClass
    +  public static void tearDown() throws Exception {
    +    clientv2.close();
    +  }
    +
    +  @Test
    +  public void testExecuteUpdate() throws TajoException {
    +    clientv2.executeUpdate("create database tajoclientv2");
    +    clientv2.selectDB("tajoclientv2");
    +    clientv2.selectDB("default");
    +    clientv2.executeUpdate("drop database tajoclientv2");
    +
    +    try {
    +      clientv2.selectDB("tajoclientv2");
    +      fail();
    +    } catch (UndefinedDatabaseException e) {
    --- End diff --
    
    selectDB is to ensure that the database is created normally and the database is dropped normally. So, I used only the exception.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968694
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java ---
    @@ -22,6 +22,7 @@
     import org.apache.tajo.catalog.FunctionDesc;
     import org.apache.tajo.datum.Datum;
     import org.apache.tajo.exception.InternalException;
    +import org.apache.tajo.exception.TajoInternalError;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968709
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/logical/SetSessionNode.java ---
    @@ -19,6 +19,7 @@
     package org.apache.tajo.plan.logical;
     
     import com.google.gson.annotations.Expose;
    +import org.apache.tajo.exception.TajoRuntimeException;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35952542
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/v2/ClientDeligateFactory.java ---
    @@ -0,0 +1,42 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +import org.apache.tajo.annotation.Nullable;
    +import org.apache.tajo.client.v2.exception.ClientUnableToConnectException;
    +
    +import java.util.Map;
    +
    +public class ClientDeligateFactory {
    +
    +  public static ClientDelegate newDefaultDeligate(String host,
    +                                                  int port,
    +                                                  @Nullable Map<String, String> props)
    +      throws ClientUnableToConnectException {
    +
    +    return new LegacyClientDelegate(host, port, props);
    +  }
    +
    +  public static ClientDelegate newDefaultDeligate(ServiceDiscovery discovery,
    --- End diff --
    
    Please fix typo.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35949195
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClient.java ---
    @@ -133,9 +135,10 @@ TableDesc createExternalTable(final String tableName, final Schema schema, final
        * @return Table description
        * @throws java.sql.SQLException
        */
    -  TableDesc getTableDesc(final String tableName) throws SQLException;
    +  TableDesc getTableDesc(final String tableName) throws UndefinedTableException;
     
    -  List<CatalogProtos.FunctionDescProto> getFunctions(final String functionName) throws SQLException;
    +  List<CatalogProtos.FunctionDescProto> getFunctions(final String functionName)
    +      throws AmbiguousFunctionException, UndefinedFunctionException;
    --- End diff --
    
    It would be better if ```getFunctions``` returns an empty list rather than throwing an ```UndefinedFunctionException``` when there is no function of the given name. 
    In addition, this method should return a list of the functions of the given name. So, ```AmbiguousFunctionException``` is inappropriate.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on the pull request:

    https://github.com/apache/tajo/pull/662#issuecomment-125879189
  
    It is ready to be reviewed.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35950685
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java ---
    @@ -21,6 +21,9 @@
     import com.google.protobuf.ServiceException;
    --- End diff --
    
    There are some unused imports. Please remove them.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968830
  
    --- Diff: tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java ---
    @@ -29,6 +29,7 @@
     import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
     import org.apache.tajo.conf.TajoConf;
     import org.apache.tajo.exception.TajoException;
    +import org.apache.tajo.exception.TajoRuntimeException;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35946742
  
    --- Diff: tajo-common/src/main/java/org/apache/tajo/exception/TajoException.java ---
    @@ -27,6 +28,16 @@
     public class TajoException extends Exception implements TajoExceptionInterface {
       private ResultCode code;
     
    +  public TajoException(ReturnState e) {
    --- End diff --
    
    ```ReturnState``` contains stack trace as well as return code and message, but the stack trace is missed here. 
    Would you give some backgrounds for this?


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on the pull request:

    https://github.com/apache/tajo/pull/662#issuecomment-127402079
  
    +1 ship it! 


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on the pull request:

    https://github.com/apache/tajo/pull/662#issuecomment-127128368
  
    I reflected the comments.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35950534
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/DummyServiceTracker.java ---
    @@ -19,6 +19,7 @@
     package org.apache.tajo.client;
     
     import org.apache.tajo.conf.TajoConf;
    +import org.apache.tajo.exception.TajoRuntimeException;
    --- End diff --
    
    Please remove the unused import.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968629
  
    --- Diff: tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java ---
    @@ -38,7 +39,7 @@
     public class TestSQLExpression extends ExprTestBase {
     
       @Test
    -  public void testQuotedIdentifiers() throws IOException {
    +  public void testQuotedIdentifiers() throws Throwable {
    --- End diff --
    
    It seems to be enough with ```TajoException``` rather than ```Throwable```.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36055977
  
    --- Diff: tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java ---
    @@ -91,7 +91,7 @@ public String getSystemDatabaseName() {
         return systemTableNames;
       }
       
    -  private TableDescriptor getTableDescriptor(String tableName) {
    +  private TableDescriptor getTableDescriptor(String tableName) throws UndefinedTableException {
    --- End diff --
    
    I'll check it. Thanks!


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968700
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerializer.java ---
    @@ -23,6 +23,7 @@
     import org.apache.tajo.common.TajoDataTypes;
     import org.apache.tajo.datum.AnyDatum;
     import org.apache.tajo.datum.Datum;
    +import org.apache.tajo.exception.TajoRuntimeException;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36056157
  
    --- Diff: tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java ---
    @@ -91,7 +91,7 @@ public String getSystemDatabaseName() {
         return systemTableNames;
       }
       
    -  private TableDescriptor getTableDescriptor(String tableName) {
    +  private TableDescriptor getTableDescriptor(String tableName) throws UndefinedTableException {
    --- End diff --
    
    According to my design, it should throw an exception instead of NULL. I fixed. Thanks!


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35948986
  
    --- Diff: tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java ---
    @@ -179,7 +179,7 @@ public AbstractDBStore(Configuration conf) throws InternalException {
               }
             }
          }
    -    } catch (Exception se) {
    +    } catch (Throwable se) {
    --- End diff --
    
    Looks good, but ```Throwable``` occurred while dropping base tables is consumed at Line 175. Please fix this too.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35952051
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/ResultSetUtil.java ---
    @@ -19,6 +19,8 @@
     package org.apache.tajo.client;
     
     import org.apache.tajo.common.TajoDataTypes;
    +import org.apache.tajo.exception.TajoInternalError;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on the pull request:

    https://github.com/apache/tajo/pull/662#issuecomment-126676275
  
    Hi @hyunsik. Thank you for the nice patch.
    I expect that this patch and your succeeding work will prevent tajo developers including me from suffering for unexpected and abused exceptions. I hope that you can finish this work nicely.
    
    I left some comments. Please consider them.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968450
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/v2/QueryFuture.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +import org.apache.tajo.auth.UserRoleInfo;
    +
    +import java.sql.ResultSet;
    +import java.util.concurrent.Future;
    +
    +public interface QueryFuture extends Future<ResultSet> {
    +  /**
    +   * Get a query id
    +   *
    +   * @return query id
    +   */
    +  String id();
    +
    +  String queue();
    +
    +  /**
    +   * Get a query state
    +   *
    +   * @return query state
    +   */
    +  QueryState state();
    +
    +  /**
    +   * Get a normalized progress (0 ~ 1.0f) of a query running
    +   *
    +   * @return progress
    +   */
    +  float progress();
    +
    +  /**
    +   * A submitted or running query state is normal
    +   *
    +   * @return True if a query state is normal
    +   */
    +  boolean isOk();
    +
    +  /**
    +   * Get whether the query is successfully completed or not.
    +   *
    +   * @return True if the query is successfully completed.
    +   */
    +  boolean isCompleted();
    --- End diff --
    
    For me, ```query is completed``` sounds that query execution is completed whether it is succeeded, failed or killed.
    How about changing ```isSucceeded()```? IMHO, it looks more intuitive.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36056280
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java ---
    @@ -94,11 +97,11 @@
        *
        * @return If failed, return null.
        */
    -  ResultSet executeQueryAndGetResult(final String sql) throws SQLException;
    +  ResultSet executeQueryAndGetResult(final String sql) throws TajoException;
    --- End diff --
    
    Take a look at the signature ``executeQuery`` and ``executeQueryWithJson``. They return SubmitQueryResponse which contains ReturnState. So, we should allow callers to handle the return state directly.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on the pull request:

    https://github.com/apache/tajo/pull/662#issuecomment-125873608
  
    rebased and fixed some unit tests.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35952656
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/v2/ClientUtil.java ---
    @@ -0,0 +1,30 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +public class ClientUtil {
    +
    +  public static boolean isOk(QueryState state) {
    --- End diff --
    
    Would you write some descriptions for these methods? It is difficult to guess its behaviour.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35949692
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java ---
    @@ -200,12 +231,17 @@ public TableDesc getTableDesc(final String tableName) throws SQLException {
           throw new RuntimeException(e);
         }
     
    -    throwIfError(res.getState());
    +    if (isThisError(res.getState(), Errors.ResultCode.UNDEFINED_TABLE)) {
    +      throw new UndefinedTableException(res.getState());
    +    }
    +
    +    ensureOk(res.getState());
         return CatalogUtil.newTableDesc(res.getTable());
       }
     
       @Override
    -  public List<FunctionDescProto> getFunctions(final String functionName) throws SQLException {
    +  public List<FunctionDescProto> getFunctions(final String functionName)
    +      throws AmbiguousFunctionException, UndefinedFunctionException {
    --- End diff --
    
    As I commented above, it would be better if ```getFunctions``` returns an empty list rather than throwing an ```UndefinedFunctionException``` when there is no function of the given name.
    In addition, this method should return a list of the functions of the given name. So, ```AmbiguousFunctionException``` is inappropriate.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36056554
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/v2/QueryFuture.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +import org.apache.tajo.auth.UserRoleInfo;
    +
    +import java.sql.ResultSet;
    +import java.util.concurrent.Future;
    +
    +public interface QueryFuture extends Future<ResultSet> {
    +  /**
    +   * Get a query id
    +   *
    +   * @return query id
    +   */
    +  String id();
    +
    +  String queue();
    +
    +  /**
    +   * Get a query state
    +   *
    +   * @return query state
    +   */
    +  QueryState state();
    +
    +  /**
    +   * Get a normalized progress (0 ~ 1.0f) of a query running
    +   *
    +   * @return progress
    +   */
    +  float progress();
    +
    +  /**
    +   * A submitted or running query state is normal
    +   *
    +   * @return True if a query state is normal
    +   */
    +  boolean isOk();
    +
    +  /**
    +   * Get whether the query is successfully completed or not.
    +   *
    +   * @return True if the query is successfully completed.
    +   */
    +  boolean isCompleted();
    --- End diff --
    
    That makes sense. I fixed.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35948192
  
    --- Diff: tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java ---
    @@ -110,7 +110,7 @@ private TableDescriptor getTableDescriptor(String tableName) {
         return tableDescriptor;
       }
       
    -  public CatalogProtos.TableDescProto getTableDesc(String tableName) {
    +  public CatalogProtos.TableDescProto getTableDesc(String tableName) throws UndefinedTableException {
    --- End diff --
    
    Both ```getTableDesc()``` and ```existTable()``` internally use ```getTableDescriptor()```, so seem to be changed according to the implementation of ```getTableDescriptor()```. Please consider the above comment.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35952528
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/v2/ClientDeligateFactory.java ---
    @@ -0,0 +1,42 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +import org.apache.tajo.annotation.Nullable;
    +import org.apache.tajo.client.v2.exception.ClientUnableToConnectException;
    +
    +import java.util.Map;
    +
    +public class ClientDeligateFactory {
    --- End diff --
    
    Please fix typo.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36056451
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/v2/ServiceDiscovery.java ---
    @@ -0,0 +1,25 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +import java.net.InetSocketAddress;
    +
    +public interface ServiceDiscovery {
    --- End diff --
    
    I gave a simple comment.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36056184
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java ---
    @@ -200,12 +231,17 @@ public TableDesc getTableDesc(final String tableName) throws SQLException {
           throw new RuntimeException(e);
         }
     
    -    throwIfError(res.getState());
    +    if (isThisError(res.getState(), Errors.ResultCode.UNDEFINED_TABLE)) {
    +      throw new UndefinedTableException(res.getState());
    +    }
    +
    +    ensureOk(res.getState());
         return CatalogUtil.newTableDesc(res.getTable());
       }
     
       @Override
    -  public List<FunctionDescProto> getFunctions(final String functionName) throws SQLException {
    +  public List<FunctionDescProto> getFunctions(final String functionName)
    +      throws AmbiguousFunctionException, UndefinedFunctionException {
    --- End diff --
    
    Fixed too.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35951781
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java ---
    @@ -94,11 +97,11 @@
        *
        * @return If failed, return null.
        */
    -  ResultSet executeQueryAndGetResult(final String sql) throws SQLException;
    +  ResultSet executeQueryAndGetResult(final String sql) throws TajoException;
    --- End diff --
    
    It is difficult to find the reason why ```executeQueryAndGetResult()``` throws ```TajoException```, but ```executeQuery()``` does not. Would you give some backgrounds?


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968704
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextFieldSerializerDeserializer.java ---
    @@ -28,6 +28,7 @@
     import org.apache.tajo.common.TajoDataTypes;
     import org.apache.tajo.datum.*;
     import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
    +import org.apache.tajo.exception.TajoInternalError;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36055958
  
    --- Diff: tajo-common/src/main/java/org/apache/tajo/exception/TajoException.java ---
    @@ -27,6 +28,16 @@
     public class TajoException extends Exception implements TajoExceptionInterface {
       private ResultCode code;
     
    +  public TajoException(ReturnState e) {
    --- End diff --
    
    Stacktrace will be used in only errors like TajoInternalError because other exceptions mean user mistake and recoverable exceptions.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968807
  
    --- Diff: tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java ---
    @@ -24,6 +24,7 @@
     import org.apache.tajo.datum.DatumFactory;
     import org.apache.tajo.datum.IntervalDatum;
     import org.apache.tajo.datum.ProtobufDatum;
    +import org.apache.tajo.exception.TajoRuntimeException;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968758
  
    --- Diff: tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NullScanner.java ---
    @@ -21,6 +21,7 @@
     import org.apache.tajo.catalog.Schema;
     import org.apache.tajo.catalog.TableMeta;
     import org.apache.tajo.catalog.statistics.TableStats;
    +import org.apache.tajo.exception.TajoRuntimeException;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36056178
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClient.java ---
    @@ -133,9 +135,10 @@ TableDesc createExternalTable(final String tableName, final Schema schema, final
        * @return Table description
        * @throws java.sql.SQLException
        */
    -  TableDesc getTableDesc(final String tableName) throws SQLException;
    +  TableDesc getTableDesc(final String tableName) throws UndefinedTableException;
     
    -  List<CatalogProtos.FunctionDescProto> getFunctions(final String functionName) throws SQLException;
    +  List<CatalogProtos.FunctionDescProto> getFunctions(final String functionName)
    +      throws AmbiguousFunctionException, UndefinedFunctionException;
    --- End diff --
    
    Fixed.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36056394
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/v2/ClientUtil.java ---
    @@ -0,0 +1,30 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +public class ClientUtil {
    +
    +  public static boolean isOk(QueryState state) {
    --- End diff --
    
    isOk means that it is neither error or failed. It allows users to know this query is running normally.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968841
  
    --- Diff: tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTuple.java ---
    @@ -24,6 +24,8 @@
     
     import org.apache.tajo.common.TajoDataTypes;
     import org.apache.tajo.datum.*;
    +import org.apache.tajo.exception.TajoInternalError;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35952857
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/v2/QueryFuture.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +import org.apache.tajo.auth.UserRoleInfo;
    +
    +import java.sql.ResultSet;
    +import java.util.concurrent.Future;
    +
    +public interface QueryFuture extends Future<ResultSet> {
    +  /**
    +   * Get a query id
    +   *
    +   * @return query id
    +   */
    +  String id();
    +
    +  String queue();
    --- End diff --
    
    Please add a description for this method.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968746
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/ProjectionPushDownRule.java ---
    @@ -30,6 +30,7 @@
     import org.apache.tajo.catalog.exception.DuplicateColumnException;
     import org.apache.tajo.exception.TajoException;
     import org.apache.tajo.exception.TajoInternalError;
    +import org.apache.tajo.exception.TajoRuntimeException;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968754
  
    --- Diff: tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java ---
    @@ -26,6 +26,7 @@
     import org.apache.tajo.datum.Datum;
     import org.apache.tajo.datum.IntervalDatum;
     import org.apache.tajo.datum.ProtobufDatum;
    +import org.apache.tajo.exception.TajoRuntimeException;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35951917
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java ---
    @@ -108,17 +111,17 @@
     
       TajoMemoryResultSet fetchNextQueryResult(final QueryId queryId, final int fetchRowNum) throws SQLException;
     
    -  boolean updateQuery(final String sql) throws SQLException;
    +  boolean updateQuery(final String sql) throws TajoException;
     
    -  boolean updateQueryWithJson(final String json) throws SQLException;
    +  boolean updateQueryWithJson(final String json) throws TajoException;
     
       List<ClientProtos.BriefQueryInfo> getRunningQueryList() throws SQLException;
    --- End diff --
    
    Some of other methods in this class still throw ```SQLException```. Is it good or do you leave them for future changes?


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968081
  
    --- Diff: tajo-core/src/test/java/org/apache/tajo/client/v2/TestTajoClientV2.java ---
    @@ -0,0 +1,236 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +import com.facebook.presto.hive.shaded.com.google.common.collect.Lists;
    +import org.apache.tajo.QueryTestCaseBase;
    +import org.apache.tajo.catalog.exception.DuplicateDatabaseException;
    +import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
    +import org.apache.tajo.catalog.exception.UndefinedTableException;
    +import org.apache.tajo.exception.TajoException;
    +import org.apache.tajo.service.ServiceTracker;
    +import org.apache.tajo.service.ServiceTrackerFactory;
    +import org.junit.AfterClass;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.io.IOException;
    +import java.net.InetSocketAddress;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.util.List;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import static org.junit.Assert.*;
    +
    +public class TestTajoClientV2 extends QueryTestCaseBase {
    +  private static TajoClient clientv2;
    +
    +  @BeforeClass
    +  public static void setUp() throws Exception {
    +    conf = testingCluster.getConfiguration();
    +
    +    clientv2 = new TajoClient(new ServiceDiscovery() {
    +      ServiceTracker tracker = ServiceTrackerFactory.get(conf);
    +      @Override
    +      public InetSocketAddress clientAddress() {
    +        return tracker.getClientServiceAddress();
    +      }
    +    });
    +  }
    +
    +  @AfterClass
    +  public static void tearDown() throws Exception {
    +    clientv2.close();
    +  }
    +
    +  @Test
    +  public void testExecuteUpdate() throws TajoException {
    +    clientv2.executeUpdate("create database tajoclientv2");
    +    clientv2.selectDB("tajoclientv2");
    +    clientv2.selectDB("default");
    +    clientv2.executeUpdate("drop database tajoclientv2");
    +
    +    try {
    +      clientv2.selectDB("tajoclientv2");
    +      fail();
    +    } catch (UndefinedDatabaseException e) {
    --- End diff --
    
    Don't we have to check other types of exceptions? 
    It seems that any exceptions except ```UndefinedDatabaseException``` are not allowed here.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35968692
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/function/AggFunctionInvoke.java ---
    @@ -23,6 +23,8 @@
     import org.apache.tajo.common.TajoDataTypes;
     import org.apache.tajo.datum.Datum;
     import org.apache.tajo.exception.InternalException;
    +import org.apache.tajo.exception.TajoInternalError;
    --- End diff --
    
    Please remove unused imports.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r35953000
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/v2/ServiceDiscovery.java ---
    @@ -0,0 +1,25 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +import java.net.InetSocketAddress;
    +
    +public interface ServiceDiscovery {
    --- End diff --
    
    Please add a description for this interface.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36056327
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java ---
    @@ -108,17 +111,17 @@
     
       TajoMemoryResultSet fetchNextQueryResult(final QueryId queryId, final int fetchRowNum) throws SQLException;
     
    -  boolean updateQuery(final String sql) throws SQLException;
    +  boolean updateQuery(final String sql) throws TajoException;
     
    -  boolean updateQueryWithJson(final String json) throws SQLException;
    +  boolean updateQueryWithJson(final String json) throws TajoException;
     
       List<ClientProtos.BriefQueryInfo> getRunningQueryList() throws SQLException;
    --- End diff --
    
    They will be changed later. I tried to avoid to change not-related codes.


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36056435
  
    --- Diff: tajo-client/src/main/java/org/apache/tajo/client/v2/QueryFuture.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.client.v2;
    +
    +import org.apache.tajo.auth.UserRoleInfo;
    +
    +import java.sql.ResultSet;
    +import java.util.concurrent.Future;
    +
    +public interface QueryFuture extends Future<ResultSet> {
    +  /**
    +   * Get a query id
    +   *
    +   * @return query id
    +   */
    +  String id();
    +
    +  String queue();
    --- End diff --
    
    added


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

[GitHub] tajo pull request: TAJO-1699: Tajo Java Client version 2.

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

    https://github.com/apache/tajo/pull/662#discussion_r36055982
  
    --- Diff: tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java ---
    @@ -110,7 +110,7 @@ private TableDescriptor getTableDescriptor(String tableName) {
         return tableDescriptor;
       }
       
    -  public CatalogProtos.TableDescProto getTableDesc(String tableName) {
    +  public CatalogProtos.TableDescProto getTableDesc(String tableName) throws UndefinedTableException {
    --- End diff --
    
    Got it.


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