You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2018/08/27 05:00:15 UTC

[GitHub] fjy closed pull request #6191: SQL: Support more result formats, add columns header.

fjy closed pull request #6191: SQL: Support more result formats, add columns header.
URL: https://github.com/apache/incubator-druid/pull/6191
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/docs/content/ingestion/index.md b/docs/content/ingestion/index.md
index 02fa1cc548e..378b308d9e9 100644
--- a/docs/content/ingestion/index.md
+++ b/docs/content/ingestion/index.md
@@ -147,7 +147,7 @@ This is a special variation of the JSON ParseSpec that lower cases all the colum
 
 #### CSV ParseSpec
 
-Use this with the String Parser to load CSV. Strings are parsed using the net.sf.opencsv library.
+Use this with the String Parser to load CSV. Strings are parsed using the com.opencsv library.
 
 | Field | Type | Description | Required |
 |-------|------|-------------|----------|
diff --git a/docs/content/querying/sql.md b/docs/content/querying/sql.md
index ffd655a76f6..0f15b936129 100644
--- a/docs/content/querying/sql.md
+++ b/docs/content/querying/sql.md
@@ -339,10 +339,6 @@ of configuration.
 You can make Druid SQL queries using JSON over HTTP by posting to the endpoint `/druid/v2/sql/`. The request should
 be a JSON object with a "query" field, like `{"query" : "SELECT COUNT(*) FROM data_source WHERE foo = 'bar'"}`.
 
-Results are available in two formats: "object" (the default; a JSON array of JSON objects), and "array" (a JSON array
-of JSON arrays). In "object" form, each row's field names will match the column names from your SQL query. In "array"
-form, each row's values are returned in the order specified in your SQL query.
-
 You can use _curl_ to send SQL queries from the command-line:
 
 ```bash
@@ -353,9 +349,8 @@ $ curl -XPOST -H'Content-Type: application/json' http://BROKER:8082/druid/v2/sql
 [{"TheCount":24433}]
 ```
 
-Metadata is available over the HTTP API by querying the ["INFORMATION_SCHEMA" tables](#retrieving-metadata).
-
-Finally, you can also provide [connection context parameters](#connection-context) by adding a "context" map, like:
+There are a variety of [connection context parameters](#connection-context) you can provide by adding a "context" map,
+like:
 
 ```json
 {
@@ -366,6 +361,45 @@ Finally, you can also provide [connection context parameters](#connection-contex
 }
 ```
 
+Metadata is available over the HTTP API by querying [system tables](#retrieving-metadata).
+
+#### Responses
+
+All Druid SQL HTTP responses include a "X-Druid-Column-Names" header with a JSON-encoded array of columns that
+will appear in the result rows and an "X-Druid-Column-Types" header with a JSON-encoded array of
+[types](#data-types-and-casts).
+
+For the result rows themselves, Druid SQL supports a variety of result formats. You can
+specify these by adding a "resultFormat" parameter, like:
+
+```json
+{
+  "query" : "SELECT COUNT(*) FROM data_source WHERE foo = 'bar' AND __time > TIMESTAMP '2000-01-01 00:00:00'",
+  "resultFormat" : "object"
+}
+```
+
+The supported result formats are:
+
+|Format|Description|Content-Type|
+|------|-----------|------------|
+|`object`|The default, a JSON array of JSON objects. Each object's field names match the columns returned by the SQL query, and are provided in the same order as the SQL query.|application/json|
+|`array`|JSON array of JSON arrays. Each inner array has elements matching the columns returned by the SQL query, in order.|application/json|
+|`objectLines`|Like "object", but the JSON objects are separated by newlines instead of being wrapped in a JSON array. This can make it easier to parse the entire response set as a stream, if you do not have ready access to a streaming JSON parser. To make it possible to detect a truncated response, this format includes a trailer of one blank line.|text/plain|
+|`arrayLines`|Like "array", but the JSON arrays are separated by newlines instead of being wrapped in a JSON array. This can make it easier to parse the entire response set as a stream, if you do not have ready access to a streaming JSON parser. To make it possible to detect a truncated response, this format includes a trailer of one blank line.|text/plain|
+|`csv`|Comma-separated values, with one row per line. Individual field values may be escaped by being surrounded in double quotes. If double quotes appear in a field value, they will be escaped by replacing them with double-double-quotes like `""this""`. To make it possible to detect a truncated response, this format includes a trailer of one blank line.|text/csv|
+
+Errors that occur before the response body is sent will be reported in JSON, with an HTTP 500 status code, in the
+same format as [native Druid query errors](../querying#query-errors). If an error occurs while the response body is
+being sent, at that point it is too late to change the HTTP status code or report a JSON error, so the response will
+simply end midstream and an error will be logged by the Druid server that was handling your request.
+
+As a caller, it is important that you properly handle response truncation. This is easy for the "object" and "array"
+formats, since truncated responses will be invalid JSON. For the line-oriented formats, you should check the
+trailer they all include: one blank line at the end of the result set. If you detect a truncated response, either
+through a JSON parsing error or through a missing trailing newline, you should assume the response was not fully
+delivered due to an error.
+
 ### JDBC
 
 You can make Druid SQL queries using the [Avatica JDBC driver](https://calcite.apache.org/avatica/downloads/). Once
@@ -398,13 +432,13 @@ Table metadata is available over JDBC using `connection.getMetaData()` or by que
 ["INFORMATION_SCHEMA" tables](#retrieving-metadata). Parameterized queries (using `?` or other placeholders) don't work properly,
 so avoid those.
 
-#### Connection Stickiness
+#### Connection stickiness
 
 Druid's JDBC server does not share connection state between brokers. This means that if you're using JDBC and have
 multiple Druid brokers, you should either connect to a specific broker, or use a load balancer with sticky sessions
-enabled.
-
-The Druid Router node provides connection stickiness when balancing JDBC requests. Please see [Router](../development/router.html) documentation for more details.
+enabled. The Druid Router node provides connection stickiness when balancing JDBC requests, and can be used to achieve
+the necessary stickiness even with a normal non-sticky load balancer. Please see the
+[Router](../development/router.html) documentation for more details.
 
 Note that the non-JDBC [JSON over HTTP](#json-over-http) API is stateless and does not require stickiness.
 
diff --git a/java-util/pom.xml b/java-util/pom.xml
index 569aa4fa8d3..4486c79a550 100644
--- a/java-util/pom.xml
+++ b/java-util/pom.xml
@@ -74,7 +74,7 @@
             <artifactId>jackson-databind</artifactId>
         </dependency>
         <dependency>
-            <groupId>net.sf.opencsv</groupId>
+            <groupId>com.opencsv</groupId>
             <artifactId>opencsv</artifactId>
         </dependency>
         <dependency>
diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/CSVParser.java b/java-util/src/main/java/io/druid/java/util/common/parsers/CSVParser.java
index e2746c3d260..3fd85b8611d 100644
--- a/java-util/src/main/java/io/druid/java/util/common/parsers/CSVParser.java
+++ b/java-util/src/main/java/io/druid/java/util/common/parsers/CSVParser.java
@@ -28,7 +28,7 @@
 
 public class CSVParser extends AbstractFlatTextFormatParser
 {
-  private final au.com.bytecode.opencsv.CSVParser parser = new au.com.bytecode.opencsv.CSVParser();
+  private final com.opencsv.CSVParser parser = new com.opencsv.CSVParser();
 
   public CSVParser(
       @Nullable final String listDelimiter,
diff --git a/pom.xml b/pom.xml
index 9774ac8b33e..83ef71a14f5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -638,9 +638,9 @@
                 <version>8.1.0</version>
             </dependency>
             <dependency>
-                <groupId>net.sf.opencsv</groupId>
+                <groupId>com.opencsv</groupId>
                 <artifactId>opencsv</artifactId>
-                <version>2.3</version>
+                <version>4.2</version>
             </dependency>
             <dependency>
                 <groupId>com.jayway.jsonpath</groupId>
diff --git a/sql/src/main/java/io/druid/sql/http/ArrayLinesWriter.java b/sql/src/main/java/io/druid/sql/http/ArrayLinesWriter.java
new file mode 100644
index 00000000000..2ff2a216a8c
--- /dev/null
+++ b/sql/src/main/java/io/druid/sql/http/ArrayLinesWriter.java
@@ -0,0 +1,81 @@
+/*
+ * 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 io.druid.sql.http;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.io.SerializedString;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.OutputStream;
+
+public class ArrayLinesWriter implements ResultFormat.Writer
+{
+  private final OutputStream outputStream;
+  private final JsonGenerator jsonGenerator;
+
+  public ArrayLinesWriter(final OutputStream outputStream, final ObjectMapper jsonMapper) throws IOException
+  {
+    this.outputStream = outputStream;
+    this.jsonGenerator = jsonMapper.writer().getFactory().createGenerator(outputStream);
+    jsonGenerator.setRootValueSeparator(new SerializedString("\n"));
+  }
+
+  @Override
+  public void writeResponseStart()
+  {
+    // Do nothing.
+  }
+
+  @Override
+  public void writeResponseEnd() throws IOException
+  {
+    jsonGenerator.flush();
+
+    // Terminate the last output line, then write an extra blank line, so users can tell the response was not cut off.
+    outputStream.write(new byte[]{'\n', '\n'});
+    outputStream.flush();
+  }
+
+  @Override
+  public void writeRowStart() throws IOException
+  {
+    jsonGenerator.writeStartArray();
+  }
+
+  @Override
+  public void writeRowField(final String name, @Nullable final Object value) throws IOException
+  {
+    jsonGenerator.writeObject(value);
+  }
+
+  @Override
+  public void writeRowEnd() throws IOException
+  {
+    jsonGenerator.writeEndArray();
+  }
+
+  @Override
+  public void close() throws IOException
+  {
+    jsonGenerator.close();
+  }
+}
diff --git a/sql/src/main/java/io/druid/sql/http/ArrayWriter.java b/sql/src/main/java/io/druid/sql/http/ArrayWriter.java
new file mode 100644
index 00000000000..dcaeb20cf3f
--- /dev/null
+++ b/sql/src/main/java/io/druid/sql/http/ArrayWriter.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 io.druid.sql.http;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.OutputStream;
+
+public class ArrayWriter implements ResultFormat.Writer
+{
+  private final JsonGenerator jsonGenerator;
+  private final OutputStream outputStream;
+
+  public ArrayWriter(final OutputStream outputStream, final ObjectMapper jsonMapper) throws IOException
+  {
+    this.jsonGenerator = jsonMapper.getFactory().createGenerator(outputStream);
+    this.outputStream = outputStream;
+  }
+
+  @Override
+  public void writeResponseStart() throws IOException
+  {
+    jsonGenerator.writeStartArray();
+  }
+
+  @Override
+  public void writeResponseEnd() throws IOException
+  {
+    jsonGenerator.writeEndArray();
+
+    // End with LF.
+    jsonGenerator.flush();
+    outputStream.write('\n');
+  }
+
+  @Override
+  public void writeRowStart() throws IOException
+  {
+    jsonGenerator.writeStartArray();
+  }
+
+  @Override
+  public void writeRowField(final String name, @Nullable final Object value) throws IOException
+  {
+    jsonGenerator.writeObject(value);
+  }
+
+  @Override
+  public void writeRowEnd() throws IOException
+  {
+    jsonGenerator.writeEndArray();
+  }
+
+  @Override
+  public void close() throws IOException
+  {
+    jsonGenerator.close();
+  }
+}
diff --git a/sql/src/main/java/io/druid/sql/http/CsvWriter.java b/sql/src/main/java/io/druid/sql/http/CsvWriter.java
new file mode 100644
index 00000000000..4858e63a850
--- /dev/null
+++ b/sql/src/main/java/io/druid/sql/http/CsvWriter.java
@@ -0,0 +1,95 @@
+/*
+ * 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 io.druid.sql.http;
+
+import com.opencsv.CSVWriter;
+
+import javax.annotation.Nullable;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+
+public class CsvWriter implements ResultFormat.Writer
+{
+  private final OutputStream outputStream;
+  private final CSVWriter writer;
+  private final List<String> currentLine = new ArrayList<>();
+
+  public CsvWriter(final OutputStream outputStream)
+  {
+    this.outputStream = outputStream;
+    this.writer = new CSVWriter(new BufferedWriter(new OutputStreamWriter(outputStream, StandardCharsets.UTF_8)));
+  }
+
+  @Override
+  public void writeResponseStart()
+  {
+    // Do nothing.
+  }
+
+  @Override
+  public void writeResponseEnd() throws IOException
+  {
+    writer.flush();
+
+    // Write an extra blank line, so users can tell the response was not cut off.
+    outputStream.write('\n');
+    outputStream.flush();
+  }
+
+  @Override
+  public void writeRowStart()
+  {
+    // Do nothing.
+  }
+
+  @Override
+  public void writeRowField(final String name, @Nullable final Object value)
+  {
+    if (value == null) {
+      // CSV cannot differentiate null and empty string.
+      currentLine.add("");
+    } else if (value instanceof String) {
+      currentLine.add((String) value);
+    } else {
+      currentLine.add(value.toString());
+    }
+  }
+
+  @Override
+  public void writeRowEnd()
+  {
+    // Avoid writing blank lines, users may confuse them with the trailer.
+    final boolean quoteEverything = currentLine.size() == 1 && currentLine.get(0).isEmpty();
+
+    writer.writeNext(currentLine.toArray(new String[0]), quoteEverything);
+    currentLine.clear();
+  }
+
+  @Override
+  public void close() throws IOException
+  {
+    writer.close();
+  }
+}
diff --git a/sql/src/main/java/io/druid/sql/http/ObjectLinesWriter.java b/sql/src/main/java/io/druid/sql/http/ObjectLinesWriter.java
new file mode 100644
index 00000000000..869d913cae6
--- /dev/null
+++ b/sql/src/main/java/io/druid/sql/http/ObjectLinesWriter.java
@@ -0,0 +1,82 @@
+/*
+ * 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 io.druid.sql.http;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.io.SerializedString;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.OutputStream;
+
+public class ObjectLinesWriter implements ResultFormat.Writer
+{
+  private final OutputStream outputStream;
+  private final JsonGenerator jsonGenerator;
+
+  public ObjectLinesWriter(final OutputStream outputStream, final ObjectMapper jsonMapper) throws IOException
+  {
+    this.outputStream = outputStream;
+    this.jsonGenerator = jsonMapper.writer().getFactory().createGenerator(outputStream);
+    jsonGenerator.setRootValueSeparator(new SerializedString("\n"));
+  }
+
+  @Override
+  public void writeResponseStart() throws IOException
+  {
+    // Do nothing.
+  }
+
+  @Override
+  public void writeResponseEnd() throws IOException
+  {
+    jsonGenerator.flush();
+
+    // Terminate the last output line, then write an extra blank line, so users can tell the response was not cut off.
+    outputStream.write(new byte[]{'\n', '\n'});
+    outputStream.flush();
+  }
+
+  @Override
+  public void writeRowStart() throws IOException
+  {
+    jsonGenerator.writeStartObject();
+  }
+
+  @Override
+  public void writeRowField(final String name, @Nullable final Object value) throws IOException
+  {
+    jsonGenerator.writeFieldName(name);
+    jsonGenerator.writeObject(value);
+  }
+
+  @Override
+  public void writeRowEnd() throws IOException
+  {
+    jsonGenerator.writeEndObject();
+  }
+
+  @Override
+  public void close() throws IOException
+  {
+    jsonGenerator.close();
+  }
+}
diff --git a/sql/src/main/java/io/druid/sql/http/ObjectWriter.java b/sql/src/main/java/io/druid/sql/http/ObjectWriter.java
new file mode 100644
index 00000000000..b247f3887e3
--- /dev/null
+++ b/sql/src/main/java/io/druid/sql/http/ObjectWriter.java
@@ -0,0 +1,80 @@
+/*
+ * 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 io.druid.sql.http;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.OutputStream;
+
+public class ObjectWriter implements ResultFormat.Writer
+{
+  private final JsonGenerator jsonGenerator;
+  private final OutputStream outputStream;
+
+  public ObjectWriter(final OutputStream outputStream, final ObjectMapper jsonMapper) throws IOException
+  {
+    this.jsonGenerator = jsonMapper.getFactory().createGenerator(outputStream);
+    this.outputStream = outputStream;
+  }
+
+  @Override
+  public void writeResponseStart() throws IOException
+  {
+    jsonGenerator.writeStartArray();
+  }
+
+  @Override
+  public void writeResponseEnd() throws IOException
+  {
+    jsonGenerator.writeEndArray();
+
+    // End with LF.
+    jsonGenerator.flush();
+    outputStream.write('\n');
+  }
+
+  @Override
+  public void writeRowStart() throws IOException
+  {
+    jsonGenerator.writeStartObject();
+  }
+
+  @Override
+  public void writeRowField(final String name, @Nullable final Object value) throws IOException
+  {
+    jsonGenerator.writeFieldName(name);
+    jsonGenerator.writeObject(value);
+  }
+
+  @Override
+  public void writeRowEnd() throws IOException
+  {
+    jsonGenerator.writeEndObject();
+  }
+
+  @Override
+  public void close() throws IOException
+  {
+    jsonGenerator.close();
+  }
+}
diff --git a/sql/src/main/java/io/druid/sql/http/ResultFormat.java b/sql/src/main/java/io/druid/sql/http/ResultFormat.java
new file mode 100644
index 00000000000..4a127d1db35
--- /dev/null
+++ b/sql/src/main/java/io/druid/sql/http/ResultFormat.java
@@ -0,0 +1,141 @@
+/*
+ * 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 io.druid.sql.http;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.druid.java.util.common.StringUtils;
+
+import javax.annotation.Nullable;
+import javax.ws.rs.core.MediaType;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.OutputStream;
+
+public enum ResultFormat
+{
+  ARRAY {
+    @Override
+    public String contentType()
+    {
+      return MediaType.APPLICATION_JSON;
+    }
+
+    @Override
+    public Writer createFormatter(final OutputStream outputStream, final ObjectMapper jsonMapper) throws IOException
+    {
+      return new ArrayWriter(outputStream, jsonMapper);
+    }
+  },
+
+  ARRAYLINES {
+    @Override
+    public String contentType()
+    {
+      return MediaType.TEXT_PLAIN;
+    }
+
+    @Override
+    public Writer createFormatter(final OutputStream outputStream, final ObjectMapper jsonMapper) throws IOException
+    {
+      return new ArrayLinesWriter(outputStream, jsonMapper);
+    }
+  },
+
+  CSV {
+    @Override
+    public String contentType()
+    {
+      return "text/csv";
+    }
+
+    @Override
+    public Writer createFormatter(final OutputStream outputStream, final ObjectMapper jsonMapper) throws IOException
+    {
+      return new CsvWriter(outputStream);
+    }
+  },
+
+  OBJECT {
+    @Override
+    public String contentType()
+    {
+      return MediaType.APPLICATION_JSON;
+    }
+
+    @Override
+    public Writer createFormatter(final OutputStream outputStream, final ObjectMapper jsonMapper) throws IOException
+    {
+      return new ObjectWriter(outputStream, jsonMapper);
+    }
+  },
+
+  OBJECTLINES {
+    @Override
+    public String contentType()
+    {
+      return MediaType.TEXT_PLAIN;
+    }
+
+    @Override
+    public Writer createFormatter(final OutputStream outputStream, final ObjectMapper jsonMapper) throws IOException
+    {
+      return new ObjectLinesWriter(outputStream, jsonMapper);
+    }
+  };
+
+  public abstract String contentType();
+
+  public abstract Writer createFormatter(OutputStream outputStream, ObjectMapper jsonMapper) throws IOException;
+
+  interface Writer extends Closeable
+  {
+    /**
+     * Start of the response, called once per writer.
+     */
+    void writeResponseStart() throws IOException;
+
+    /**
+     * Start of each result row.
+     */
+    void writeRowStart() throws IOException;
+
+    /**
+     * Field within a row.
+     */
+    void writeRowField(String name, @Nullable Object value) throws IOException;
+
+    /**
+     * End of each result row.
+     */
+    void writeRowEnd() throws IOException;
+
+    /**
+     * End of the response. Must allow the user to know that they have read all data successfully.
+     */
+    void writeResponseEnd() throws IOException;
+  }
+
+  @JsonCreator
+  public static ResultFormat fromString(final String name)
+  {
+    return valueOf(StringUtils.toUpperCase(name));
+  }
+}
diff --git a/sql/src/main/java/io/druid/sql/http/SqlQuery.java b/sql/src/main/java/io/druid/sql/http/SqlQuery.java
index 31c53480d72..de0dd41d0e8 100644
--- a/sql/src/main/java/io/druid/sql/http/SqlQuery.java
+++ b/sql/src/main/java/io/druid/sql/http/SqlQuery.java
@@ -21,86 +21,14 @@
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonGenerator;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
-import io.druid.java.util.common.StringUtils;
 
-import javax.annotation.Nullable;
-import java.io.IOException;
 import java.util.Map;
 import java.util.Objects;
 
 public class SqlQuery
 {
-  public enum ResultFormat
-  {
-    ARRAY {
-      @Override
-      public void writeResultStart(final JsonGenerator jsonGenerator) throws IOException
-      {
-        jsonGenerator.writeStartArray();
-      }
-
-      @Override
-      public void writeResultField(
-          final JsonGenerator jsonGenerator,
-          final String name,
-          final Object value
-      ) throws IOException
-      {
-        jsonGenerator.writeObject(value);
-      }
-
-      @Override
-      public void writeResultEnd(final JsonGenerator jsonGenerator) throws IOException
-      {
-        jsonGenerator.writeEndArray();
-      }
-    },
-
-    OBJECT {
-      @Override
-      public void writeResultStart(final JsonGenerator jsonGenerator) throws IOException
-      {
-        jsonGenerator.writeStartObject();
-      }
-
-      @Override
-      public void writeResultField(
-          final JsonGenerator jsonGenerator,
-          final String name,
-          final Object value
-      ) throws IOException
-      {
-        jsonGenerator.writeFieldName(name);
-        jsonGenerator.writeObject(value);
-      }
-
-      @Override
-      public void writeResultEnd(final JsonGenerator jsonGenerator) throws IOException
-      {
-        jsonGenerator.writeEndObject();
-      }
-    };
-
-    public abstract void writeResultStart(JsonGenerator jsonGenerator) throws IOException;
-
-    public abstract void writeResultField(JsonGenerator jsonGenerator, String name, Object value)
-        throws IOException;
-
-    public abstract void writeResultEnd(JsonGenerator jsonGenerator) throws IOException;
-
-    @JsonCreator
-    public static ResultFormat fromString(@Nullable final String name)
-    {
-      if (name == null) {
-        return null;
-      }
-      return valueOf(StringUtils.toUpperCase(name));
-    }
-  }
-
   private final String query;
   private final ResultFormat resultFormat;
   private final Map<String, Object> context;
diff --git a/sql/src/main/java/io/druid/sql/http/SqlResource.java b/sql/src/main/java/io/druid/sql/http/SqlResource.java
index cd26db749ac..fa89e790bc5 100644
--- a/sql/src/main/java/io/druid/sql/http/SqlResource.java
+++ b/sql/src/main/java/io/druid/sql/http/SqlResource.java
@@ -19,7 +19,6 @@
 
 package io.druid.sql.http;
 
-import com.fasterxml.jackson.core.JsonGenerator;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
@@ -88,66 +87,72 @@ public Response doPost(
       timeZone = planner.getPlannerContext().getTimeZone();
 
       // Remember which columns are time-typed, so we can emit ISO8601 instead of millis values.
+      // Also store list of all column names, for X-Druid-Sql-Columns header.
       final List<RelDataTypeField> fieldList = plannerResult.rowType().getFieldList();
       final boolean[] timeColumns = new boolean[fieldList.size()];
       final boolean[] dateColumns = new boolean[fieldList.size()];
+      final String[] columnNames = new String[fieldList.size()];
+      final String[] columnTypes = new String[fieldList.size()];
+
       for (int i = 0; i < fieldList.size(); i++) {
         final SqlTypeName sqlTypeName = fieldList.get(i).getType().getSqlTypeName();
         timeColumns[i] = sqlTypeName == SqlTypeName.TIMESTAMP;
         dateColumns[i] = sqlTypeName == SqlTypeName.DATE;
+        columnNames[i] = fieldList.get(i).getName();
+        columnTypes[i] = sqlTypeName.getName();
       }
 
       final Yielder<Object[]> yielder0 = Yielders.each(plannerResult.run());
 
       try {
-        return Response.ok(
-            new StreamingOutput()
-            {
-              @Override
-              public void write(final OutputStream outputStream) throws IOException, WebApplicationException
-              {
-                Yielder<Object[]> yielder = yielder0;
-
-                try (final JsonGenerator jsonGenerator = jsonMapper.getFactory().createGenerator(outputStream)) {
-                  jsonGenerator.writeStartArray();
-
-                  while (!yielder.isDone()) {
-                    final Object[] row = yielder.get();
-                    sqlQuery.getResultFormat().writeResultStart(jsonGenerator);
-                    for (int i = 0; i < fieldList.size(); i++) {
-                      final Object value;
-
-                      if (timeColumns[i]) {
-                        value = ISODateTimeFormat.dateTime().print(
-                            Calcites.calciteTimestampToJoda((long) row[i], timeZone)
-                        );
-                      } else if (dateColumns[i]) {
-                        value = ISODateTimeFormat.dateTime().print(
-                            Calcites.calciteDateToJoda((int) row[i], timeZone)
-                        );
-                      } else {
-                        value = row[i];
+        return Response
+            .ok(
+                new StreamingOutput()
+                {
+                  @Override
+                  public void write(final OutputStream outputStream) throws IOException, WebApplicationException
+                  {
+                    Yielder<Object[]> yielder = yielder0;
+
+                    try (final ResultFormat.Writer writer = sqlQuery.getResultFormat()
+                                                                    .createFormatter(outputStream, jsonMapper)) {
+                      writer.writeResponseStart();
+
+                      while (!yielder.isDone()) {
+                        final Object[] row = yielder.get();
+                        writer.writeRowStart();
+                        for (int i = 0; i < fieldList.size(); i++) {
+                          final Object value;
+
+                          if (timeColumns[i]) {
+                            value = ISODateTimeFormat.dateTime().print(
+                                Calcites.calciteTimestampToJoda((long) row[i], timeZone)
+                            );
+                          } else if (dateColumns[i]) {
+                            value = ISODateTimeFormat.dateTime().print(
+                                Calcites.calciteDateToJoda((int) row[i], timeZone)
+                            );
+                          } else {
+                            value = row[i];
+                          }
+
+                          writer.writeRowField(fieldList.get(i).getName(), value);
+                        }
+                        writer.writeRowEnd();
+                        yielder = yielder.next(null);
                       }
 
-                      sqlQuery.getResultFormat().writeResultField(jsonGenerator, fieldList.get(i).getName(), value);
+                      writer.writeResponseEnd();
+                    }
+                    finally {
+                      yielder.close();
                     }
-                    sqlQuery.getResultFormat().writeResultEnd(jsonGenerator);
-                    yielder = yielder.next(null);
                   }
-
-                  jsonGenerator.writeEndArray();
-                  jsonGenerator.flush();
-
-                  // End with CRLF
-                  outputStream.write('\r');
-                  outputStream.write('\n');
-                }
-                finally {
-                  yielder.close();
                 }
-              }
-            }
-        ).build();
+            )
+            .header("X-Druid-Column-Names", jsonMapper.writeValueAsString(columnNames))
+            .header("X-Druid-Column-Types", jsonMapper.writeValueAsString(columnTypes))
+            .build();
       }
       catch (Throwable e) {
         // make sure to close yielder if anything happened before starting to serialize the response.
diff --git a/sql/src/test/java/io/druid/sql/calcite/http/SqlQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/http/SqlQueryTest.java
index 397467c07ab..fb8e876b4b4 100644
--- a/sql/src/test/java/io/druid/sql/calcite/http/SqlQueryTest.java
+++ b/sql/src/test/java/io/druid/sql/calcite/http/SqlQueryTest.java
@@ -23,6 +23,7 @@
 import com.google.common.collect.ImmutableMap;
 import io.druid.segment.TestHelper;
 import io.druid.sql.calcite.util.CalciteTestBase;
+import io.druid.sql.http.ResultFormat;
 import io.druid.sql.http.SqlQuery;
 import org.junit.Assert;
 import org.junit.Test;
@@ -33,7 +34,7 @@
   public void testSerde() throws Exception
   {
     final ObjectMapper jsonMapper = TestHelper.makeJsonMapper();
-    final SqlQuery query = new SqlQuery("SELECT 1", SqlQuery.ResultFormat.ARRAY, ImmutableMap.of("useCache", false));
+    final SqlQuery query = new SqlQuery("SELECT 1", ResultFormat.ARRAY, ImmutableMap.of("useCache", false));
     Assert.assertEquals(query, jsonMapper.readValue(jsonMapper.writeValueAsString(query), SqlQuery.class));
   }
 }
diff --git a/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java b/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java
index 849c8eb7193..b5e15a97048 100644
--- a/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java
+++ b/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java
@@ -21,6 +21,7 @@
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Splitter;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
@@ -45,6 +46,7 @@
 import io.druid.sql.calcite.util.CalciteTests;
 import io.druid.sql.calcite.util.QueryLogHook;
 import io.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
+import io.druid.sql.http.ResultFormat;
 import io.druid.sql.http.SqlQuery;
 import io.druid.sql.http.SqlResource;
 import org.apache.calcite.tools.ValidationException;
@@ -63,8 +65,12 @@
 import javax.ws.rs.core.StreamingOutput;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
 
 public class SqlResourceTest extends CalciteTestBase
 {
@@ -146,33 +152,38 @@ public void tearDown() throws Exception
   }
 
   @Test
-  public void testCountStar() throws Exception
+  public void testXDruidColumnHeaders() throws Exception
   {
-    final List<Map<String, Object>> rows = doPost(
-        new SqlQuery("SELECT COUNT(*) AS cnt, 'foo' AS TheFoo FROM druid.foo", null, null)
-    ).rhs;
+    final Response response = resource.doPost(
+        new SqlQuery(
+            "SELECT FLOOR(__time TO DAY) as \"day\", COUNT(*) as TheCount, SUM(m1) FROM druid.foo GROUP BY 1",
+            ResultFormat.OBJECT,
+            null
+        ),
+        req
+    );
 
     Assert.assertEquals(
-        ImmutableList.of(
-            ImmutableMap.of("cnt", 6, "TheFoo", "foo")
-        ),
-        rows
+        "[\"day\",\"TheCount\",\"EXPR$2\"]",
+        response.getMetadata().getFirst("X-Druid-Column-Names")
+    );
+
+    Assert.assertEquals(
+        "[\"TIMESTAMP\",\"BIGINT\",\"DOUBLE\"]",
+        response.getMetadata().getFirst("X-Druid-Column-Types")
     );
   }
 
   @Test
-  public void testCountStarAsArray() throws Exception
+  public void testCountStar() throws Exception
   {
-    final List<List<Object>> rows = doPost(
-        new SqlQuery("SELECT COUNT(*), 'foo' FROM druid.foo", SqlQuery.ResultFormat.ARRAY, null),
-        new TypeReference<List<List<Object>>>()
-        {
-        }
+    final List<Map<String, Object>> rows = doPost(
+        new SqlQuery("SELECT COUNT(*) AS cnt, 'foo' AS TheFoo FROM druid.foo", null, null)
     ).rhs;
 
     Assert.assertEquals(
         ImmutableList.of(
-            ImmutableList.of(6, "foo")
+            ImmutableMap.of("cnt", 6, "TheFoo", "foo")
         ),
         rows
     );
@@ -184,7 +195,7 @@ public void testTimestampsInResponse() throws Exception
     final List<Map<String, Object>> rows = doPost(
         new SqlQuery(
             "SELECT __time, CAST(__time AS DATE) AS t2 FROM druid.foo LIMIT 1",
-            SqlQuery.ResultFormat.OBJECT,
+            ResultFormat.OBJECT,
             null
         )
     ).rhs;
@@ -203,7 +214,7 @@ public void testTimestampsInResponseLosAngelesTimeZone() throws Exception
     final List<Map<String, Object>> rows = doPost(
         new SqlQuery(
             "SELECT __time, CAST(__time AS DATE) AS t2 FROM druid.foo LIMIT 1",
-            SqlQuery.ResultFormat.OBJECT,
+            ResultFormat.OBJECT,
             ImmutableMap.of(PlannerContext.CTX_SQL_TIME_ZONE, "America/Los_Angeles")
         )
     ).rhs;
@@ -220,7 +231,7 @@ public void testTimestampsInResponseLosAngelesTimeZone() throws Exception
   public void testFieldAliasingSelect() throws Exception
   {
     final List<Map<String, Object>> rows = doPost(
-        new SqlQuery("SELECT dim2 \"x\", dim2 \"y\" FROM druid.foo LIMIT 1", SqlQuery.ResultFormat.OBJECT, null)
+        new SqlQuery("SELECT dim2 \"x\", dim2 \"y\" FROM druid.foo LIMIT 1", ResultFormat.OBJECT, null)
     ).rhs;
 
     Assert.assertEquals(
@@ -235,7 +246,7 @@ public void testFieldAliasingSelect() throws Exception
   public void testFieldAliasingGroupBy() throws Exception
   {
     final List<Map<String, Object>> rows = doPost(
-        new SqlQuery("SELECT dim2 \"x\", dim2 \"y\" FROM druid.foo GROUP BY dim2", SqlQuery.ResultFormat.OBJECT, null)
+        new SqlQuery("SELECT dim2 \"x\", dim2 \"y\" FROM druid.foo GROUP BY dim2", ResultFormat.OBJECT, null)
     ).rhs;
 
     Assert.assertEquals(
@@ -256,11 +267,157 @@ public void testFieldAliasingGroupBy() throws Exception
     );
   }
 
+  @Test
+  public void testArrayResultFormat() throws Exception
+  {
+    final String query = "SELECT *, CASE dim2 WHEN '' THEN dim2 END FROM foo LIMIT 2";
+    final String nullStr = NullHandling.replaceWithDefault() ? "" : null;
+
+    Assert.assertEquals(
+        ImmutableList.of(
+            Arrays.asList("2000-01-01T00:00:00.000Z", 1, "", "a", 1.0, 1.0, "io.druid.hll.HLLCV1", nullStr),
+            Arrays.asList("2000-01-02T00:00:00.000Z", 1, "10.1", nullStr, 2.0, 2.0, "io.druid.hll.HLLCV1", nullStr)
+        ),
+        doPost(new SqlQuery(query, ResultFormat.ARRAY, null), new TypeReference<List<List<Object>>>() {}).rhs
+    );
+  }
+
+  @Test
+  public void testArrayLinesResultFormat() throws Exception
+  {
+    final String query = "SELECT *, CASE dim2 WHEN '' THEN dim2 END FROM foo LIMIT 2";
+    final String response = doPostRaw(new SqlQuery(query, ResultFormat.ARRAYLINES, null)).rhs;
+    final String nullStr = NullHandling.replaceWithDefault() ? "" : null;
+    final List<String> lines = Splitter.on('\n').splitToList(response);
+
+    Assert.assertEquals(4, lines.size());
+    Assert.assertEquals(
+        Arrays.asList("2000-01-01T00:00:00.000Z", 1, "", "a", 1.0, 1.0, "io.druid.hll.HLLCV1", nullStr),
+        JSON_MAPPER.readValue(lines.get(0), List.class)
+    );
+    Assert.assertEquals(
+        Arrays.asList("2000-01-02T00:00:00.000Z", 1, "10.1", nullStr, 2.0, 2.0, "io.druid.hll.HLLCV1", nullStr),
+        JSON_MAPPER.readValue(lines.get(1), List.class)
+    );
+    Assert.assertEquals("", lines.get(2));
+    Assert.assertEquals("", lines.get(3));
+  }
+
+  @Test
+  public void testObjectResultFormat() throws Exception
+  {
+    final String query = "SELECT *, CASE dim2 WHEN '' THEN dim2 END FROM foo  LIMIT 2";
+    final String nullStr = NullHandling.replaceWithDefault() ? "" : null;
+    final Function<Map<String, Object>, Map<String, Object>> transformer = m -> {
+      return Maps.transformEntries(
+          m,
+          (k, v) -> "EXPR$7".equals(k) || ("dim2".equals(k) && v.toString().isEmpty()) ? nullStr : v
+      );
+    };
+
+    Assert.assertEquals(
+        ImmutableList.of(
+            ImmutableMap
+                .<String, Object>builder()
+                .put("__time", "2000-01-01T00:00:00.000Z")
+                .put("cnt", 1)
+                .put("dim1", "")
+                .put("dim2", "a")
+                .put("m1", 1.0)
+                .put("m2", 1.0)
+                .put("unique_dim1", "io.druid.hll.HLLCV1")
+                .put("EXPR$7", "")
+                .build(),
+            ImmutableMap
+                .<String, Object>builder()
+                .put("__time", "2000-01-02T00:00:00.000Z")
+                .put("cnt", 1)
+                .put("dim1", "10.1")
+                .put("dim2", "")
+                .put("m1", 2.0)
+                .put("m2", 2.0)
+                .put("unique_dim1", "io.druid.hll.HLLCV1")
+                .put("EXPR$7", "")
+                .build()
+        ).stream().map(transformer).collect(Collectors.toList()),
+        doPost(new SqlQuery(query, ResultFormat.OBJECT, null), new TypeReference<List<Map<String, Object>>>() {}).rhs
+    );
+  }
+
+  @Test
+  public void testObjectLinesResultFormat() throws Exception
+  {
+    final String query = "SELECT *, CASE dim2 WHEN '' THEN dim2 END FROM foo LIMIT 2";
+    final String response = doPostRaw(new SqlQuery(query, ResultFormat.OBJECTLINES, null)).rhs;
+    final String nullStr = NullHandling.replaceWithDefault() ? "" : null;
+    final Function<Map<String, Object>, Map<String, Object>> transformer = m -> {
+      return Maps.transformEntries(
+          m,
+          (k, v) -> "EXPR$7".equals(k) || ("dim2".equals(k) && v.toString().isEmpty()) ? nullStr : v
+      );
+    };
+    final List<String> lines = Splitter.on('\n').splitToList(response);
+
+    Assert.assertEquals(4, lines.size());
+    Assert.assertEquals(
+        transformer.apply(
+            ImmutableMap
+                .<String, Object>builder()
+                .put("__time", "2000-01-01T00:00:00.000Z")
+                .put("cnt", 1)
+                .put("dim1", "")
+                .put("dim2", "a")
+                .put("m1", 1.0)
+                .put("m2", 1.0)
+                .put("unique_dim1", "io.druid.hll.HLLCV1")
+                .put("EXPR$7", "")
+                .build()
+        ),
+        JSON_MAPPER.readValue(lines.get(0), Object.class)
+    );
+    Assert.assertEquals(
+        transformer.apply(
+            ImmutableMap
+                .<String, Object>builder()
+                .put("__time", "2000-01-02T00:00:00.000Z")
+                .put("cnt", 1)
+                .put("dim1", "10.1")
+                .put("dim2", "")
+                .put("m1", 2.0)
+                .put("m2", 2.0)
+                .put("unique_dim1", "io.druid.hll.HLLCV1")
+                .put("EXPR$7", "")
+                .build()
+        ),
+        JSON_MAPPER.readValue(lines.get(1), Object.class)
+    );
+    Assert.assertEquals("", lines.get(2));
+    Assert.assertEquals("", lines.get(3));
+  }
+
+  @Test
+  public void testCsvResultFormat() throws Exception
+  {
+    final String query = "SELECT *, CASE dim2 WHEN '' THEN dim2 END FROM foo LIMIT 2";
+    final String response = doPostRaw(new SqlQuery(query, ResultFormat.CSV, null)).rhs;
+    final List<String> lines = Splitter.on('\n').splitToList(response);
+
+    Assert.assertEquals(
+        ImmutableList.of(
+            "2000-01-01T00:00:00.000Z,1,,a,1.0,1.0,io.druid.hll.HLLCV1,",
+            "2000-01-02T00:00:00.000Z,1,10.1,,2.0,2.0,io.druid.hll.HLLCV1,",
+            "",
+            ""
+        ),
+        lines
+    );
+  }
+
   @Test
   public void testExplainCountStar() throws Exception
   {
     final List<Map<String, Object>> rows = doPost(
-        new SqlQuery("EXPLAIN PLAN FOR SELECT COUNT(*) AS cnt FROM druid.foo", SqlQuery.ResultFormat.OBJECT, null)
+        new SqlQuery("EXPLAIN PLAN FOR SELECT COUNT(*) AS cnt FROM druid.foo", ResultFormat.OBJECT, null)
     ).rhs;
 
     Assert.assertEquals(
@@ -280,7 +437,7 @@ public void testCannotValidate() throws Exception
     final QueryInterruptedException exception = doPost(
         new SqlQuery(
             "SELECT dim3 FROM druid.foo",
-            SqlQuery.ResultFormat.OBJECT,
+            ResultFormat.OBJECT,
             null
         )
     ).lhs;
@@ -296,7 +453,7 @@ public void testCannotConvert() throws Exception
   {
     // SELECT + ORDER unsupported
     final QueryInterruptedException exception = doPost(
-        new SqlQuery("SELECT dim1 FROM druid.foo ORDER BY dim1", SqlQuery.ResultFormat.OBJECT, null)
+        new SqlQuery("SELECT dim1 FROM druid.foo ORDER BY dim1", ResultFormat.OBJECT, null)
     ).lhs;
 
     Assert.assertNotNull(exception);
@@ -314,7 +471,7 @@ public void testResourceLimitExceeded() throws Exception
     final QueryInterruptedException exception = doPost(
         new SqlQuery(
             "SELECT DISTINCT dim1 FROM foo",
-            SqlQuery.ResultFormat.OBJECT,
+            ResultFormat.OBJECT,
             ImmutableMap.of("maxMergingDictionarySize", 1)
         )
     ).lhs;
@@ -324,11 +481,23 @@ public void testResourceLimitExceeded() throws Exception
     Assert.assertEquals(exception.getErrorClass(), ResourceLimitExceededException.class.getName());
   }
 
-  // Returns either an error or a result.
+  // Returns either an error or a result, assuming the result is a JSON object.
   private <T> Pair<QueryInterruptedException, T> doPost(
       final SqlQuery query,
       final TypeReference<T> typeReference
   ) throws Exception
+  {
+    final Pair<QueryInterruptedException, String> pair = doPostRaw(query);
+    if (pair.rhs == null) {
+      //noinspection unchecked
+      return (Pair<QueryInterruptedException, T>) pair;
+    } else {
+      return Pair.of(pair.lhs, JSON_MAPPER.readValue(pair.rhs, typeReference));
+    }
+  }
+
+  // Returns either an error or a result.
+  private Pair<QueryInterruptedException, String> doPostRaw(final SqlQuery query) throws Exception
   {
     final Response response = resource.doPost(query, req);
     if (response.getStatus() == 200) {
@@ -337,7 +506,7 @@ public void testResourceLimitExceeded() throws Exception
       output.write(baos);
       return Pair.of(
           null,
-          JSON_MAPPER.readValue(baos.toByteArray(), typeReference)
+          new String(baos.toByteArray(), StandardCharsets.UTF_8)
       );
     } else {
       return Pair.of(


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org