You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2015/04/16 23:02:27 UTC

[3/6] incubator-calcite git commit: [CALCITE-671] ByteString does not deserialize properly as a FetchRequest parameterValue

[CALCITE-671] ByteString does not deserialize properly as a FetchRequest parameterValue

Add class TypedValue, which encodes to JSON as say {type: LONG, value: 1}, to provide information about how the value should be decoded to a Java type (in this case, Long).

Encode BINARY and VARBINARY parameters and column values as Base64 strings as they go over JSON, convert ByteString in Avatica code, then to byte[] in JDBC.

Move RhBase64 to Base64 under Avatica and upgrade to version 2.3.7.

Fix "expirydiration" typo.


Project: http://git-wip-us.apache.org/repos/asf/incubator-calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-calcite/commit/3e26a873
Tree: http://git-wip-us.apache.org/repos/asf/incubator-calcite/tree/3e26a873
Diff: http://git-wip-us.apache.org/repos/asf/incubator-calcite/diff/3e26a873

Branch: refs/heads/master
Commit: 3e26a873ae17759a9fce12d56c133488558fb771
Parents: 8276f25
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Apr 7 23:46:36 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Apr 16 02:21:42 2015 -0700

----------------------------------------------------------------------
 .../apache/calcite/avatica/jdbc/JdbcMeta.java   |   10 +-
 .../calcite/avatica/RemoteDriverTest.java       |   22 +
 .../apache/calcite/avatica/ColumnMetaData.java  |    3 +
 .../calcite/avatica/remote/JsonService.java     |    7 +
 .../calcite/avatica/remote/LocalService.java    |    2 +-
 .../calcite/avatica/remote/RemoteMeta.java      |    3 +-
 .../apache/calcite/avatica/remote/Service.java  |    4 +-
 .../calcite/avatica/remote/TypedValue.java      |  127 ++
 .../calcite/avatica/util/AbstractCursor.java    |   34 +-
 .../org/apache/calcite/avatica/util/Base64.java | 2087 ++++++++++++++++++
 .../apache/calcite/avatica/util/ByteString.java |   96 +
 .../java/org/apache/calcite/util/RhBase64.java  | 1391 ------------
 .../apache/calcite/test/SqlFunctionsTest.java   |   14 +
 13 files changed, 2401 insertions(+), 1399 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e26a873/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
index 3b241de..72274ff 100644
--- a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
+++ b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
@@ -20,6 +20,7 @@ import org.apache.calcite.avatica.AvaticaParameter;
 import org.apache.calcite.avatica.ColumnMetaData;
 import org.apache.calcite.avatica.ConnectionPropertiesImpl;
 import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.util.ByteString;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -102,7 +103,7 @@ public class JdbcMeta implements Meta {
     MAX_CAPACITY(CONN_CACHE_KEY_BASE + ".maxcapacity", "1000"),
 
     /** JDBC connection property for setting connection cache expiration duration. */
-    EXPIRY_DURATION(CONN_CACHE_KEY_BASE + ".expirydiration", "10"),
+    EXPIRY_DURATION(CONN_CACHE_KEY_BASE + ".expiryduration", "10"),
 
     /** JDBC connection property for setting connection cache expiration unit. */
     EXPIRY_UNIT(CONN_CACHE_KEY_BASE + ".expiryunit", TimeUnit.MINUTES.name());
@@ -154,7 +155,7 @@ public class JdbcMeta implements Meta {
     private final String key;
     private final String defaultValue;
 
-    private StatementCacheSettings(String key, String defaultValue) {
+    StatementCacheSettings(String key, String defaultValue) {
       this.key = key;
       this.defaultValue = defaultValue;
     }
@@ -306,7 +307,7 @@ public class JdbcMeta implements Meta {
 
   /**
    * @param url a database url of the form
-   *  <code> jdbc:<em>subprotocol</em>:<em>subname</em></code>
+   *  <code>jdbc:<em>subprotocol</em>:<em>subname</em></code>
    */
   public JdbcMeta(String url) throws SQLException {
     this(url, new Properties());
@@ -805,6 +806,9 @@ public class JdbcMeta implements Meta {
         if (parameterValues != null) {
           for (int i = 0; i < parameterValues.size(); i++) {
             Object o = parameterValues.get(i);
+            if (o instanceof ByteString) {
+              o = ((ByteString) o).getBytes();
+            }
             preparedStatement.setObject(i + 1, o);
           }
         }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e26a873/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
index 1c0e7d9..f221f99 100644
--- a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
+++ b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
@@ -469,6 +469,28 @@ public class RemoteDriverTest {
     connection.close();
   }
 
+  @Test public void testPrepareBindExecuteFetch2() throws Exception {
+    checkPrepareBindExecuteFetch2(ljs());
+  }
+
+  private void checkPrepareBindExecuteFetch2(Connection connection)
+      throws SQLException {
+    final String sql = "select x'de' || ? as c from (values (1, 'a'))";
+    final PreparedStatement ps =
+        connection.prepareStatement(sql);
+    final ParameterMetaData parameterMetaData = ps.getParameterMetaData();
+    assertThat(parameterMetaData.getParameterCount(), equalTo(1));
+
+    ps.setBytes(1, new byte[] {65, 0, 66});
+    final ResultSet resultSet = ps.executeQuery();
+    assertTrue(resultSet.next());
+    assertThat(resultSet.getBytes(1),
+        equalTo(new byte[] {(byte) 0xDE, 65, 0, 66}));
+    resultSet.close();
+    ps.close();
+    connection.close();
+  }
+
   /**
    * Factory that creates a service based on a local JDBC connection.
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e26a873/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java b/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
index ed03710..9f35277 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.avatica;
 
+import org.apache.calcite.avatica.util.ByteString;
+
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonSubTypes;
@@ -190,6 +192,7 @@ public class ColumnMetaData {
     JAVA_SQL_TIMESTAMP(Timestamp.class),
     JAVA_SQL_DATE(java.sql.Date.class),
     JAVA_UTIL_DATE(java.util.Date.class),
+    BYTE_STRING(ByteString.class),
     STRING(String.class),
     OBJECT(Object.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e26a873/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
index d8e22e6..4be898e 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
@@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 
 import java.io.IOException;
 import java.io.StringWriter;
+import java.sql.Types;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -70,6 +71,12 @@ public abstract class JsonService implements Service {
         column = column.setRep(ColumnMetaData.Rep.OBJECT);
         ++changeCount;
       }
+      switch (column.type.id) {
+      case Types.VARBINARY:
+      case Types.BINARY:
+        column = column.setRep(ColumnMetaData.Rep.STRING);
+        ++changeCount;
+      }
       columns.add(column);
     }
     if (changeCount == 0) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e26a873/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
index aad40e1..f35265c 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
@@ -161,7 +161,7 @@ public class LocalService implements Service {
         request.connectionId, request.statementId, null);
     final Meta.Frame frame =
         meta.fetch(h,
-            request.parameterValues,
+            TypedValue.values(request.parameterValues),
             request.offset,
             request.fetchMaxRowCount);
     return new FetchResponse(frame);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e26a873/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
index 9a3eab6..8a83ff9 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
@@ -175,9 +175,10 @@ class RemoteMeta extends MetaImpl {
 
   @Override public Frame fetch(StatementHandle h, List<Object> parameterValues,
       int offset, int fetchMaxRowCount) {
+    final List<TypedValue> typedValues = TypedValue.list(parameterValues);
     final Service.FetchResponse response =
         service.apply(
-            new Service.FetchRequest(h.connectionId, h.id, parameterValues,
+            new Service.FetchRequest(h.connectionId, h.id, typedValues,
                 offset, fetchMaxRowCount));
     return response.frame;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e26a873/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
index 430450d..3e9f24f 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
@@ -294,13 +294,13 @@ public interface Service {
     public final int fetchMaxRowCount;
     /** A list of parameter values, if statement is to be executed; otherwise
      * null. */
-    public final List<Object> parameterValues;
+    public final List<TypedValue> parameterValues;
 
     @JsonCreator
     public FetchRequest(
         @JsonProperty("connectionId") String connectionId,
         @JsonProperty("statementId") int statementId,
-        @JsonProperty("parameterValues") List<Object> parameterValues,
+        @JsonProperty("parameterValues") List<TypedValue> parameterValues,
         @JsonProperty("offset") int offset,
         @JsonProperty("fetchMaxRowCount") int fetchMaxRowCount) {
       this.connectionId = connectionId;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e26a873/avatica/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
new file mode 100644
index 0000000..6ccb443
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
@@ -0,0 +1,127 @@
+/*
+ * 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.calcite.avatica.remote;
+
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.util.ByteString;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Value and type. */
+public class TypedValue {
+  /** Type of the value. */
+  public final ColumnMetaData.Rep type;
+
+  /** Value.
+   *
+   * <p>Always in a form that can be serialized to JSON by Jackson.
+   * For example, byte arrays are represented as String. */
+  public final Object value;
+
+  private TypedValue(ColumnMetaData.Rep rep, Object value) {
+    this.type = rep;
+    this.value = serialize(rep, value);
+  }
+
+  @JsonCreator
+  public static TypedValue create(@JsonProperty("type") String type,
+      @JsonProperty("value") Object value) {
+    ColumnMetaData.Rep rep = ColumnMetaData.Rep.valueOf(type);
+    return new TypedValue(rep, deserialize(rep, value));
+  }
+
+  /** Converts the value into the preferred representation.
+   *
+   * <p>For example, a byte string is represented as a {@link ByteString};
+   * a long is represented as a {@link Long} (not just some {@link Number}).
+   */
+  public Object deserialize() {
+    return deserialize(type, value);
+  }
+
+  /** Converts a value to the exact type required for the given
+   * representation. */
+  private static Object deserialize(ColumnMetaData.Rep rep, Object value) {
+    if (value == null) {
+      return null;
+    }
+    if (value.getClass() == rep.clazz) {
+      return value;
+    }
+    switch (rep) {
+    case BYTE:
+      return ((Number) value).byteValue();
+    case SHORT:
+      return ((Number) value).shortValue();
+    case INTEGER:
+      return ((Number) value).intValue();
+    case LONG:
+      return ((Number) value).longValue();
+    case FLOAT:
+      return ((Number) value).floatValue();
+    case DOUBLE:
+      return ((Number) value).doubleValue();
+    case BYTE_STRING:
+      return ByteString.ofBase64((String) value);
+    default:
+      throw new IllegalArgumentException("cannot convert " + value + " ("
+          + value.getClass() + ") to " + rep);
+    }
+  }
+
+  /** Converts a value to a type that can be serialized as JSON. */
+  private static Object serialize(ColumnMetaData.Rep rep, Object value) {
+    switch (rep) {
+    case BYTE_STRING:
+      return ((ByteString) value).toBase64String();
+    default:
+      return value;
+    }
+  }
+
+  /** Converts a list of {@code TypedValue} to a list of values. */
+  public static List<Object> values(List<TypedValue> typedValues) {
+    final List<Object> list = new ArrayList<>();
+    for (TypedValue typedValue : typedValues) {
+      list.add(typedValue.deserialize());
+    }
+    return list;
+  }
+
+  /** Converts a list of values to a list of {@code TypedValue}. */
+  public static List<TypedValue> list(List<Object> values) {
+    final List<TypedValue> list = new ArrayList<>();
+    for (Object value : values) {
+      list.add(create(value));
+    }
+    return list;
+  }
+
+  private static TypedValue create(Object value) {
+    if (value == null) {
+      return new TypedValue(ColumnMetaData.Rep.OBJECT, null);
+    }
+    final ColumnMetaData.Rep rep = ColumnMetaData.Rep.of(value.getClass());
+    return new TypedValue(rep, value);
+  }
+}
+
+// End TypedValue.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e26a873/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java b/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
index e1c9f75..a910c3d 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
@@ -122,7 +122,12 @@ public abstract class AbstractCursor implements Cursor {
       return new StringAccessor(getter);
     case Types.BINARY:
     case Types.VARBINARY:
-      return new BinaryAccessor(getter);
+      switch (columnMetaData.type.rep) {
+      case STRING:
+        return new BinaryFromStringAccessor(getter);
+      default:
+        return new BinaryAccessor(getter);
+      }
     case Types.DATE:
       switch (columnMetaData.type.rep) {
       case PRIMITIVE_INT:
@@ -681,6 +686,10 @@ public abstract class AbstractCursor implements Cursor {
     public String getString() {
       return (String) getObject();
     }
+
+    @Override public byte[] getBytes() {
+      return super.getBytes();
+    }
   }
 
   /**
@@ -740,6 +749,29 @@ public abstract class AbstractCursor implements Cursor {
   }
 
   /**
+   * Accessor that assumes that the underlying value is a {@link String},
+   * encoding {@link java.sql.Types#BINARY}
+   * and {@link java.sql.Types#VARBINARY} values in Base64 format.
+   */
+  private static class BinaryFromStringAccessor extends StringAccessor {
+    public BinaryFromStringAccessor(Getter getter) {
+      super(getter);
+    }
+
+    @Override public Object getObject() {
+      return super.getObject();
+    }
+
+    @Override public byte[] getBytes() {
+      final String string = getString();
+      if (string == null) {
+        return null;
+      }
+      return ByteString.parseBase64(string);
+    }
+  }
+
+  /**
    * Accessor that assumes that the underlying value is a DATE,
    * in its default representation {@code int};
    * corresponds to {@link java.sql.Types#DATE}.