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

[GitHub] tajo pull request: TAJO-1944: Support text resultset for REST

GitHub user charsyam opened a pull request:

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

    TAJO-1944: Support text resultset for REST

    This is just for review. and I just add csv only now :)
    
    I worried about 3 points. 
    
    first of all, memory usages for other type.
    I think most users want to download all data as csv one time.
    
    second, add new method for NonForwardQueryResultScanner to get tuple list.
    
    third, removing base64 checking for consistency with other type(csv, binary)

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

    $ git pull https://github.com/charsyam/tajo feature/tajo-output

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

    https://github.com/apache/tajo/pull/876.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 #876
    
----

----


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r47099223
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/annotation/RestReturnType.java ---
    @@ -0,0 +1,34 @@
    +/**
    + * 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.ws.rs.annotation;
    +
    +import java.lang.annotation.ElementType;
    +import java.lang.annotation.Retention;
    +import java.lang.annotation.RetentionPolicy;
    +import java.lang.annotation.Target;
    +
    +/**
    + * Definition of RestApi result Return Type
    + * According to headerType, StramingOutput class will be choose.
    --- End diff --
    
    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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46091561
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/CSVStreamingOutput.java ---
    @@ -0,0 +1,134 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.ws.rs.resources.outputs;
    +
    +import com.google.protobuf.ByteString;
    +import org.apache.commons.lang.StringEscapeUtils;
    +import org.apache.tajo.catalog.Column;
    +import org.apache.tajo.catalog.Schema;
    +import org.apache.tajo.common.TajoDataTypes;
    +import org.apache.tajo.datum.Datum;
    +import org.apache.tajo.engine.function.annotation.ParamTypes;
    +import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
    +import org.apache.tajo.storage.Tuple;
    +import org.apache.tajo.ws.rs.annotation.RestReturnType;
    +
    +import javax.ws.rs.WebApplicationException;
    +import javax.ws.rs.core.MediaType;
    +import java.io.BufferedOutputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.List;
    +
    +@RestReturnType(
    +        description = "default binary streaming output",
    +        headerType = "application/csv"
    +)
    +public class CSVStreamingOutput extends AbstractStreamingOutput {
    +    private List<Tuple> outpuTupletList;
    --- End diff --
    
    The list of tuples doesn't have to be kept. Removing this list will reduce the memory usage.


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

[GitHub] tajo pull request: TAJO-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r47455679
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/RestOutputFactory.java ---
    @@ -0,0 +1,85 @@
    +/**
    + * 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.ws.rs.resources.outputs;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
    +import org.apache.tajo.util.ClassUtil;
    +import org.apache.tajo.ws.rs.annotation.RestReturnType;
    +
    +import java.lang.reflect.Modifier;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class RestOutputFactory {
    +  private static Log LOG = LogFactory.getLog(RestOutputFactory.class);
    +  private static Map<String, String> restOutputClasses = load();
    +
    +  private static Map<String, String> load() {
    +    Map<String, String> outputClasses = Maps.newHashMap();
    +    Set<Class> restOutputClasses = ClassUtil.findClasses(AbstractStreamingOutput.class, "org.apache.tajo.ws.rs.resources.outputs");
    +
    +    for (Class eachClass : restOutputClasses) {
    +      if (eachClass.isInterface() || 
    +          Modifier.isAbstract(eachClass.getModifiers())) {
    +        continue;
    +      }
    +
    +      AbstractStreamingOutput streamingOutput = null;
    +      try {
    +        streamingOutput = (AbstractStreamingOutput) eachClass.getDeclaredConstructor(
    +        new Class[]{NonForwardQueryResultScanner.class, Integer.class, Integer.class}).newInstance(null, 0, 0);
    +      } catch (Exception e) {
    +        LOG.warn(eachClass + " cannot instantiate Function class because of " + e.getMessage(), e);
    +        continue;
    +      }
    +      String className = streamingOutput.getClass().getCanonicalName();
    +      String headerType = streamingOutput.getClass().getAnnotation(RestReturnType.class).headerType();
    +
    +      if (StringUtils.isNotEmpty(headerType)) {
    +        outputClasses.put(headerType, className);
    +      }
    +    }
    +
    +    return outputClasses;
    +  }
    +
    +  public static AbstractStreamingOutput get(String headerType, NonForwardQueryResultScanner scanner, Integer count, Integer startOffset) {
    +    AbstractStreamingOutput output = null;
    +    try {
    +      if (restOutputClasses.containsKey(headerType)) {
    +        String className = (String) restOutputClasses.get(headerType);
    +        Class<?> clazz = Class.forName(className);
    +        output = (AbstractStreamingOutput) clazz.getDeclaredConstructor(
    +                  new Class[]{NonForwardQueryResultScanner.class,
    +                  Integer.class, Integer.class})
    +                  .newInstance(scanner, count, startOffset);
    +      } else {
    +        output = new BinaryStreamingOutput(scanner, count, startOffset);
    --- End diff --
    
    From that point, I think it will be ok because BinaryStreamingOutput is designed to be used for clients of other languages. 


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#issuecomment-167048985
  
    +1. The latest patch looks good to me.
    Please remove the above unused import before commit.
    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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#issuecomment-164474417
  
    @jihoonson I applied your reivews Thanks :)
    1. changing headerType -> mimeType
    2. changing default output type to "CSV"
    3. fix test and Description.
    
    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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46093935
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/CSVStreamingOutput.java ---
    @@ -0,0 +1,134 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.ws.rs.resources.outputs;
    +
    +import com.google.protobuf.ByteString;
    +import org.apache.commons.lang.StringEscapeUtils;
    +import org.apache.tajo.catalog.Column;
    +import org.apache.tajo.catalog.Schema;
    +import org.apache.tajo.common.TajoDataTypes;
    +import org.apache.tajo.datum.Datum;
    +import org.apache.tajo.engine.function.annotation.ParamTypes;
    +import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
    +import org.apache.tajo.storage.Tuple;
    +import org.apache.tajo.ws.rs.annotation.RestReturnType;
    +
    +import javax.ws.rs.WebApplicationException;
    +import javax.ws.rs.core.MediaType;
    +import java.io.BufferedOutputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.List;
    +
    +@RestReturnType(
    +        description = "default binary streaming output",
    +        headerType = "application/csv"
    +)
    +public class CSVStreamingOutput extends AbstractStreamingOutput {
    +    private List<Tuple> outpuTupletList;
    +    private String output;
    +    private boolean alreadyCalculated = false;
    +    private int length = 0;
    +
    +    public CSVStreamingOutput(NonForwardQueryResultScanner cachedQueryResultScanner, Integer count, Integer startOffset) throws IOException {
    +        super(cachedQueryResultScanner, count, startOffset);
    +    }
    +
    +    @Override
    +    public boolean hasLength() {
    +        return true;
    +    }
    +
    +    @Override
    +    public int length() {
    +        try {
    +            fetch();
    +            return output.length();
    +        } catch (Exception e) {
    +            return 0;
    +        }
    +    }
    +
    +    @Override
    +    public int count() {
    +        try {
    +            fetch();
    +            return outpuTupletList.size();
    +        } catch (Exception e) {
    +            return 0;
    +        }
    +    }
    +
    +    private void fetch() throws IOException {
    +        if (output != null) {
    +            return;
    +        }
    +
    +        outpuTupletList = scanner.getNextTupleRows(count);
    +
    +        StringBuilder sb = new StringBuilder();
    +        if (startOffset == 0) {
    +            Schema schema = this.scanner.getLogicalSchema();
    +            List<Column> columns = schema.getAllColumns();
    +            boolean first = true;
    +            for (Column column : columns) {
    +                if (first == false) {
    --- End diff --
    
    @jihoonson you mean just replace first == false -> !first
    is it coding convention of tajo?
    Actually, I prefer using this explict expression
    
    but it is coding convention of tajo. I will replace :) 


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r47066412
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/annotation/RestReturnType.java ---
    @@ -0,0 +1,34 @@
    +/**
    + * 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.ws.rs.annotation;
    +
    +import java.lang.annotation.ElementType;
    +import java.lang.annotation.Retention;
    +import java.lang.annotation.RetentionPolicy;
    +import java.lang.annotation.Target;
    +
    +/**
    + * Definition of RestApi result Return Type
    + * According to headerType, StramingOutput class will be choose.
    --- End diff --
    
    How about to change like below?
    ```
    Definition of the return type of RestAPI.
    According to the output type, RestAPIs return their results in text or binary.
    ```


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46091585
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/RestOutputFactory.java ---
    @@ -0,0 +1,91 @@
    +/**
    + * 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.ws.rs.resources.outputs;
    +
    +import com.facebook.presto.hive.shaded.com.google.common.collect.Maps;
    --- End diff --
    
    Please use google's library instead of facebook's one.


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#issuecomment-160416125
  
    Hi @charsyam, I left some 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-1944: Support text resultset for REST

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

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


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r47098780
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/annotation/RestReturnType.java ---
    @@ -0,0 +1,34 @@
    +/**
    + * 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.ws.rs.annotation;
    +
    +import java.lang.annotation.ElementType;
    +import java.lang.annotation.Retention;
    +import java.lang.annotation.RetentionPolicy;
    +import java.lang.annotation.Target;
    +
    +/**
    + * Definition of RestApi result Return Type
    + * According to headerType, StramingOutput class will be choose.
    + */
    +@Retention(RetentionPolicy.RUNTIME)
    +@Target(ElementType.TYPE)
    +public @interface RestReturnType {
    +    String headerType();
    --- End diff --
    
    @jihoonson how about mimeType? event though, It is type for output.
    but, It has mime type like "application/csv" or "application/octet-stream"


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46091572
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/BinaryStreamingOutput.java ---
    @@ -0,0 +1,79 @@
    +/**
    + * 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.ws.rs.resources.outputs;
    +
    +import com.google.protobuf.ByteString;
    +import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
    +import org.apache.tajo.ws.rs.annotation.RestReturnType;
    +
    +import javax.ws.rs.WebApplicationException;
    +import java.io.BufferedOutputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.List;
    +
    +@RestReturnType(
    +        description = "default binary streaming output",
    +        headerType = "application/octet-stream"
    +)
    +public class BinaryStreamingOutput extends AbstractStreamingOutput {
    +    private List<ByteString> outputList = null;
    +
    +    public BinaryStreamingOutput(NonForwardQueryResultScanner scanner, Integer count, Integer startOffset) throws IOException {
    +        super(scanner, count, startOffset);
    +    }
    +
    +    @Override
    +    public boolean hasLength() {
    +        return false;
    --- End diff --
    
    Binary output length can also be calculated.


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46091886
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/RestOutputFactory.java ---
    @@ -0,0 +1,91 @@
    +/**
    + * 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.ws.rs.resources.outputs;
    +
    +import com.facebook.presto.hive.shaded.com.google.common.collect.Maps;
    +import com.facebook.presto.hive.shaded.org.apache.commons.lang.StringUtils;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.tajo.catalog.FunctionDesc;
    +import org.apache.tajo.common.TajoDataTypes;
    +import org.apache.tajo.conf.TajoConf;
    +import org.apache.tajo.engine.function.annotation.Description;
    +import org.apache.tajo.engine.function.annotation.ParamTypes;
    +import org.apache.tajo.function.Function;
    +import org.apache.tajo.function.FunctionSignature;
    +import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
    +import org.apache.tajo.util.ClassUtil;
    +import org.apache.tajo.ws.rs.annotation.RestReturnType;
    +
    +import java.lang.reflect.Constructor;
    +import java.lang.reflect.Modifier;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class RestOutputFactory {
    +    private static Log LOG = LogFactory.getLog(RestOutputFactory.class);
    +    private static Map<String, String> restOutputClasses = load();
    +
    +    private static Map<String, String> load() {
    +        Map<String, String> outputClasses = Maps.newHashMap();
    +        Set<Class> restOutputClasses = ClassUtil.findClasses(AbstractStreamingOutput.class, "org.apache.tajo.ws.rs.resources.outputs");
    +
    +        for (Class eachClass : restOutputClasses) {
    +            if (eachClass.isInterface() || Modifier.isAbstract(eachClass.getModifiers())) {
    +                continue;
    +            }
    +
    +            AbstractStreamingOutput streamingOutput = null;
    +            try {
    +                streamingOutput = (AbstractStreamingOutput) eachClass.getDeclaredConstructor(
    --- End diff --
    
    ```streamingOutput``` instance is created unnecessarily.


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#issuecomment-159781809
  
    I'm reviewing 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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#issuecomment-163160353
  
    @charsyam thank you for updating patch. I left some trivial comments.
    In addition, please fix the test failure.
    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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46091582
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/RestOutputFactory.java ---
    @@ -0,0 +1,91 @@
    +/**
    + * 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.ws.rs.resources.outputs;
    +
    +import com.facebook.presto.hive.shaded.com.google.common.collect.Maps;
    +import com.facebook.presto.hive.shaded.org.apache.commons.lang.StringUtils;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.tajo.catalog.FunctionDesc;
    --- 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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r47066504
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/annotation/RestReturnType.java ---
    @@ -0,0 +1,34 @@
    +/**
    + * 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.ws.rs.annotation;
    +
    +import java.lang.annotation.ElementType;
    +import java.lang.annotation.Retention;
    +import java.lang.annotation.RetentionPolicy;
    +import java.lang.annotation.Target;
    +
    +/**
    + * Definition of RestApi result Return Type
    + * According to headerType, StramingOutput class will be choose.
    + */
    +@Retention(RetentionPolicy.RUNTIME)
    +@Target(ElementType.TYPE)
    +public @interface RestReturnType {
    +    String headerType();
    --- End diff --
    
    ```header type``` sounds less intuitive to me. How about ```output type```?


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

[GitHub] tajo pull request: TAJO-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r47099213
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/RestOutputFactory.java ---
    @@ -0,0 +1,85 @@
    +/**
    + * 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.ws.rs.resources.outputs;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
    +import org.apache.tajo.util.ClassUtil;
    +import org.apache.tajo.ws.rs.annotation.RestReturnType;
    +
    +import java.lang.reflect.Modifier;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class RestOutputFactory {
    +  private static Log LOG = LogFactory.getLog(RestOutputFactory.class);
    +  private static Map<String, String> restOutputClasses = load();
    +
    +  private static Map<String, String> load() {
    +    Map<String, String> outputClasses = Maps.newHashMap();
    +    Set<Class> restOutputClasses = ClassUtil.findClasses(AbstractStreamingOutput.class, "org.apache.tajo.ws.rs.resources.outputs");
    +
    +    for (Class eachClass : restOutputClasses) {
    +      if (eachClass.isInterface() || 
    +          Modifier.isAbstract(eachClass.getModifiers())) {
    +        continue;
    +      }
    +
    +      AbstractStreamingOutput streamingOutput = null;
    +      try {
    +        streamingOutput = (AbstractStreamingOutput) eachClass.getDeclaredConstructor(
    +        new Class[]{NonForwardQueryResultScanner.class, Integer.class, Integer.class}).newInstance(null, 0, 0);
    +      } catch (Exception e) {
    +        LOG.warn(eachClass + " cannot instantiate Function class because of " + e.getMessage(), e);
    +        continue;
    +      }
    +      String className = streamingOutput.getClass().getCanonicalName();
    +      String headerType = streamingOutput.getClass().getAnnotation(RestReturnType.class).headerType();
    +
    +      if (StringUtils.isNotEmpty(headerType)) {
    +        outputClasses.put(headerType, className);
    +      }
    +    }
    +
    +    return outputClasses;
    +  }
    +
    +  public static AbstractStreamingOutput get(String headerType, NonForwardQueryResultScanner scanner, Integer count, Integer startOffset) {
    +    AbstractStreamingOutput output = null;
    +    try {
    +      if (restOutputClasses.containsKey(headerType)) {
    +        String className = (String) restOutputClasses.get(headerType);
    +        Class<?> clazz = Class.forName(className);
    +        output = (AbstractStreamingOutput) clazz.getDeclaredConstructor(
    +                  new Class[]{NonForwardQueryResultScanner.class,
    +                  Integer.class, Integer.class})
    +                  .newInstance(scanner, count, startOffset);
    +      } else {
    +        output = new BinaryStreamingOutput(scanner, count, startOffset);
    --- End diff --
    
    @jihoonson I think it is better. but I also afraid of changing default output type because some users are already using it with binary type. What do you think? 


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

[GitHub] tajo pull request: TAJO-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46094222
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/BinaryStreamingOutput.java ---
    @@ -0,0 +1,79 @@
    +/**
    + * 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.ws.rs.resources.outputs;
    +
    +import com.google.protobuf.ByteString;
    +import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
    +import org.apache.tajo.ws.rs.annotation.RestReturnType;
    +
    +import javax.ws.rs.WebApplicationException;
    +import java.io.BufferedOutputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.List;
    +
    +@RestReturnType(
    +        description = "default binary streaming output",
    +        headerType = "application/octet-stream"
    +)
    +public class BinaryStreamingOutput extends AbstractStreamingOutput {
    +    private List<ByteString> outputList = null;
    +
    +    public BinaryStreamingOutput(NonForwardQueryResultScanner scanner, Integer count, Integer startOffset) throws IOException {
    +        super(scanner, count, startOffset);
    +    }
    +
    +    @Override
    +    public boolean hasLength() {
    +        return false;
    --- End diff --
    
    @jihoonson Yes, we can.
    But, I think BinaryStramingOutput class send bytestring with size.
    so I think calculating length is not needed.
    What do you think?


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

[GitHub] tajo pull request: TAJO-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r48397759
  
    --- Diff: tajo-core-tests/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java ---
    @@ -26,6 +26,7 @@
     import org.apache.tajo.exception.ErrorUtil;
     import org.apache.tajo.storage.RowStoreUtil;
     import org.apache.tajo.storage.Tuple;
    +import org.apache.tajo.storage.thirdparty.orc.IntegerColumnStatistics;
    --- End diff --
    
    Unnecessary 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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46091734
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/annotation/RestReturnType.java ---
    @@ -0,0 +1,37 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.ws.rs.annotation;
    +
    +import java.lang.annotation.ElementType;
    +import java.lang.annotation.Retention;
    +import java.lang.annotation.RetentionPolicy;
    +import java.lang.annotation.Target;
    +
    +/**
    + * Description.
    + *
    + */
    +@Retention(RetentionPolicy.RUNTIME)
    +@Target(ElementType.TYPE)
    +public @interface RestReturnType {
    +
    +    String description() default "";
    --- End diff --
    
    ```Description``` seems to be not used. Maybe you have a plan to use it later.
    Would you share your intention? 


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46091443
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/CSVStreamingOutput.java ---
    @@ -0,0 +1,134 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.ws.rs.resources.outputs;
    +
    +import com.google.protobuf.ByteString;
    --- 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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46093999
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/annotation/RestReturnType.java ---
    @@ -0,0 +1,37 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.ws.rs.annotation;
    +
    +import java.lang.annotation.ElementType;
    +import java.lang.annotation.Retention;
    +import java.lang.annotation.RetentionPolicy;
    +import java.lang.annotation.Target;
    +
    +/**
    + * Description.
    + *
    + */
    +@Retention(RetentionPolicy.RUNTIME)
    +@Target(ElementType.TYPE)
    +public @interface RestReturnType {
    +
    +    String description() default "";
    --- End diff --
    
    Actually, I think someone that want to give description need.
    so, I think currently removing description is better :)
    Thanks. I will remove it soon.


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46107725
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/BinaryStreamingOutput.java ---
    @@ -0,0 +1,79 @@
    +/**
    + * 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.ws.rs.resources.outputs;
    +
    +import com.google.protobuf.ByteString;
    +import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
    +import org.apache.tajo.ws.rs.annotation.RestReturnType;
    +
    +import javax.ws.rs.WebApplicationException;
    +import java.io.BufferedOutputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.List;
    +
    +@RestReturnType(
    +        description = "default binary streaming output",
    +        headerType = "application/octet-stream"
    +)
    +public class BinaryStreamingOutput extends AbstractStreamingOutput {
    +    private List<ByteString> outputList = null;
    +
    +    public BinaryStreamingOutput(NonForwardQueryResultScanner scanner, Integer count, Integer startOffset) throws IOException {
    +        super(scanner, count, startOffset);
    +    }
    +
    +    @Override
    +    public boolean hasLength() {
    +        return false;
    --- End diff --
    
    When BinaryStreamingOutput is used, Content-Length attribute of the http header can be filled by calculating the total length of the binary data. Content-Length attribute needs to be filled in most cases. Please see http://www.w3.org/Protocols/rfc2616/rfc2616-sec14.html


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46091808
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/CSVStreamingOutput.java ---
    @@ -0,0 +1,134 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.ws.rs.resources.outputs;
    +
    +import com.google.protobuf.ByteString;
    +import org.apache.commons.lang.StringEscapeUtils;
    +import org.apache.tajo.catalog.Column;
    +import org.apache.tajo.catalog.Schema;
    +import org.apache.tajo.common.TajoDataTypes;
    +import org.apache.tajo.datum.Datum;
    +import org.apache.tajo.engine.function.annotation.ParamTypes;
    +import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
    +import org.apache.tajo.storage.Tuple;
    +import org.apache.tajo.ws.rs.annotation.RestReturnType;
    +
    +import javax.ws.rs.WebApplicationException;
    +import javax.ws.rs.core.MediaType;
    +import java.io.BufferedOutputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.List;
    +
    +@RestReturnType(
    +        description = "default binary streaming output",
    +        headerType = "application/csv"
    +)
    +public class CSVStreamingOutput extends AbstractStreamingOutput {
    +    private List<Tuple> outpuTupletList;
    +    private String output;
    +    private boolean alreadyCalculated = false;
    --- End diff --
    
    Please remove unused variables.


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r47066950
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/RestOutputFactory.java ---
    @@ -0,0 +1,85 @@
    +/**
    + * 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.ws.rs.resources.outputs;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
    +import org.apache.tajo.util.ClassUtil;
    +import org.apache.tajo.ws.rs.annotation.RestReturnType;
    +
    +import java.lang.reflect.Modifier;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class RestOutputFactory {
    +  private static Log LOG = LogFactory.getLog(RestOutputFactory.class);
    +  private static Map<String, String> restOutputClasses = load();
    +
    +  private static Map<String, String> load() {
    +    Map<String, String> outputClasses = Maps.newHashMap();
    +    Set<Class> restOutputClasses = ClassUtil.findClasses(AbstractStreamingOutput.class, "org.apache.tajo.ws.rs.resources.outputs");
    +
    +    for (Class eachClass : restOutputClasses) {
    +      if (eachClass.isInterface() || 
    +          Modifier.isAbstract(eachClass.getModifiers())) {
    +        continue;
    +      }
    +
    +      AbstractStreamingOutput streamingOutput = null;
    +      try {
    +        streamingOutput = (AbstractStreamingOutput) eachClass.getDeclaredConstructor(
    +        new Class[]{NonForwardQueryResultScanner.class, Integer.class, Integer.class}).newInstance(null, 0, 0);
    +      } catch (Exception e) {
    +        LOG.warn(eachClass + " cannot instantiate Function class because of " + e.getMessage(), e);
    +        continue;
    +      }
    +      String className = streamingOutput.getClass().getCanonicalName();
    +      String headerType = streamingOutput.getClass().getAnnotation(RestReturnType.class).headerType();
    +
    +      if (StringUtils.isNotEmpty(headerType)) {
    +        outputClasses.put(headerType, className);
    +      }
    +    }
    +
    +    return outputClasses;
    +  }
    +
    +  public static AbstractStreamingOutput get(String headerType, NonForwardQueryResultScanner scanner, Integer count, Integer startOffset) {
    +    AbstractStreamingOutput output = null;
    +    try {
    +      if (restOutputClasses.containsKey(headerType)) {
    +        String className = (String) restOutputClasses.get(headerType);
    +        Class<?> clazz = Class.forName(className);
    +        output = (AbstractStreamingOutput) clazz.getDeclaredConstructor(
    +                  new Class[]{NonForwardQueryResultScanner.class,
    +                  Integer.class, Integer.class})
    +                  .newInstance(scanner, count, startOffset);
    +      } else {
    +        output = new BinaryStreamingOutput(scanner, count, startOffset);
    --- End diff --
    
    I think most users are familiar with TextStreamingOutput. How about to use it as the default encoding format?


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#issuecomment-165388401
  
    Thanks, but some tests are still failed. Please check it.


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

[GitHub] tajo pull request: TAJO-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#issuecomment-159766586
  
    The patch looks good to me. Soon, I'll finish the review.


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46091442
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/AbstractStreamingOutput.java ---
    @@ -0,0 +1,46 @@
    +/**
    + * 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.ws.rs.resources.outputs;
    +
    +import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
    +
    +import javax.ws.rs.core.HttpHeaders;
    --- End diff --
    
    Please remove 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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46091467
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/CSVStreamingOutput.java ---
    @@ -0,0 +1,134 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tajo.ws.rs.resources.outputs;
    +
    +import com.google.protobuf.ByteString;
    +import org.apache.commons.lang.StringEscapeUtils;
    +import org.apache.tajo.catalog.Column;
    +import org.apache.tajo.catalog.Schema;
    +import org.apache.tajo.common.TajoDataTypes;
    +import org.apache.tajo.datum.Datum;
    +import org.apache.tajo.engine.function.annotation.ParamTypes;
    +import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
    +import org.apache.tajo.storage.Tuple;
    +import org.apache.tajo.ws.rs.annotation.RestReturnType;
    +
    +import javax.ws.rs.WebApplicationException;
    +import javax.ws.rs.core.MediaType;
    +import java.io.BufferedOutputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.List;
    +
    +@RestReturnType(
    +        description = "default binary streaming output",
    +        headerType = "application/csv"
    +)
    +public class CSVStreamingOutput extends AbstractStreamingOutput {
    +    private List<Tuple> outpuTupletList;
    +    private String output;
    +    private boolean alreadyCalculated = false;
    +    private int length = 0;
    +
    +    public CSVStreamingOutput(NonForwardQueryResultScanner cachedQueryResultScanner, Integer count, Integer startOffset) throws IOException {
    +        super(cachedQueryResultScanner, count, startOffset);
    +    }
    +
    +    @Override
    +    public boolean hasLength() {
    +        return true;
    +    }
    +
    +    @Override
    +    public int length() {
    +        try {
    +            fetch();
    +            return output.length();
    +        } catch (Exception e) {
    +            return 0;
    +        }
    +    }
    +
    +    @Override
    +    public int count() {
    +        try {
    +            fetch();
    +            return outpuTupletList.size();
    +        } catch (Exception e) {
    +            return 0;
    +        }
    +    }
    +
    +    private void fetch() throws IOException {
    +        if (output != null) {
    +            return;
    +        }
    +
    +        outpuTupletList = scanner.getNextTupleRows(count);
    +
    +        StringBuilder sb = new StringBuilder();
    +        if (startOffset == 0) {
    +            Schema schema = this.scanner.getLogicalSchema();
    +            List<Column> columns = schema.getAllColumns();
    +            boolean first = true;
    +            for (Column column : columns) {
    +                if (first == false) {
    --- End diff --
    
    This condition can be simplified to ```!first```.


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r46093972
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/outputs/RestOutputFactory.java ---
    @@ -0,0 +1,91 @@
    +/**
    + * 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.ws.rs.resources.outputs;
    +
    +import com.facebook.presto.hive.shaded.com.google.common.collect.Maps;
    --- End diff --
    
    Thanks. Actually, I missed 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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#issuecomment-161355813
  
    @jihoonson I added length for BinaryStreamingOutput.
    but I can't change output style of BinaryStreamingOutput for Compatibility :) Thanks for your review :)


---
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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#issuecomment-165808854
  
    @jihoonson I think fail is because of travis ci's OOM.
    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-1944: Support text resultset for REST

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

    https://github.com/apache/tajo/pull/876#discussion_r47455608
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/ws/rs/annotation/RestReturnType.java ---
    @@ -0,0 +1,34 @@
    +/**
    + * 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.ws.rs.annotation;
    +
    +import java.lang.annotation.ElementType;
    +import java.lang.annotation.Retention;
    +import java.lang.annotation.RetentionPolicy;
    +import java.lang.annotation.Target;
    +
    +/**
    + * Definition of RestApi result Return Type
    + * According to headerType, StramingOutput class will be choose.
    + */
    +@Retention(RetentionPolicy.RUNTIME)
    +@Target(ElementType.TYPE)
    +public @interface RestReturnType {
    +    String headerType();
    --- End diff --
    
    Well, mimeType also looks good. If you prefer to mimeType, ```application/csv``` needs to be changed to ```text/csv```. Please refer to http://www.freeformatter.com/mime-types-list.html#mime-types-list.


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

[GitHub] tajo pull request: TAJO-1944: Support text resultset for REST

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

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


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