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/09/17 02:10:53 UTC

[7/8] incubator-calcite git commit: [CALCITE-840] Protocol buffer serialization over HTTP for Avatica Server (Josh Elser)

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/cb7c213c/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/Meta.java b/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
index fe61aaf..e0162b3 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.avatica;
 
+import org.apache.calcite.avatica.proto.Common;
+import org.apache.calcite.avatica.remote.ProtobufService;
 import org.apache.calcite.avatica.remote.TypedValue;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
@@ -23,14 +25,20 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor;
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
+import java.math.BigDecimal;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -335,6 +343,14 @@ public interface Meta {
       }
       return aClass.cast(v);
     }
+
+    public static DatabaseProperty fromProto(Common.DatabaseProperty proto) {
+      return DatabaseProperty.valueOf(proto.getName());
+    }
+
+    public Common.DatabaseProperty toProto() {
+      return Common.DatabaseProperty.newBuilder().setName(name()).build();
+    }
   }
 
   /** Response from execute.
@@ -480,6 +496,86 @@ public interface Meta {
         return ARRAY;
       }
     }
+
+    public Common.CursorFactory toProto() {
+      Common.CursorFactory.Builder builder = Common.CursorFactory.newBuilder();
+
+      if (null != clazz) {
+        builder.setClassName(clazz.getName());
+      }
+      builder.setStyle(style.toProto());
+      if (null != fieldNames) {
+        builder.addAllFieldNames(fieldNames);
+      }
+
+      return builder.build();
+    }
+
+    public static CursorFactory fromProto(Common.CursorFactory proto) {
+      // Reconstruct CursorFactory
+      Class<?> clz = null;
+
+      FieldDescriptor clzFieldDesc = proto.getDescriptorForType()
+          .findFieldByNumber(Common.CursorFactory.CLASS_NAME_FIELD_NUMBER);
+
+      if (proto.hasField(clzFieldDesc)) {
+        try {
+          clz = Class.forName(proto.getClassName());
+        } catch (ClassNotFoundException e) {
+          throw new RuntimeException(e);
+        }
+      }
+
+      return CursorFactory.create(Style.fromProto(proto.getStyle()), clz,
+          proto.getFieldNamesList());
+    }
+
+    @Override public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((clazz == null) ? 0 : clazz.hashCode());
+      result = prime * result + ((fieldNames == null) ? 0 : fieldNames.hashCode());
+      result = prime * result + ((fields == null) ? 0 : fields.hashCode());
+      result = prime * result + ((style == null) ? 0 : style.hashCode());
+      return result;
+    }
+
+    @Override public boolean equals(Object o) {
+      if (o == this) {
+        return true;
+      }
+      if (o instanceof CursorFactory) {
+        CursorFactory other = (CursorFactory) o;
+
+        if (null == clazz) {
+          if (null != other.clazz) {
+            return false;
+          }
+        } else if (!clazz.equals(other.clazz)) {
+          return false;
+        }
+
+        if (null == fieldNames) {
+          if (null != other.fieldNames) {
+            return false;
+          }
+        } else if (!fieldNames.equals(other.fieldNames)) {
+          return false;
+        }
+
+        if (null == fields) {
+          if (null != other.fields) {
+            return false;
+          }
+        } else if (!fields.equals(other.fields)) {
+          return false;
+        }
+
+        return style == other.style;
+      }
+
+      return false;
+    }
   }
 
   /** How logical fields are represented in the objects returned by the
@@ -490,11 +586,19 @@ public interface Meta {
     RECORD_PROJECTION,
     ARRAY,
     LIST,
-    MAP
+    MAP;
+
+    public Common.CursorFactory.Style toProto() {
+      return Common.CursorFactory.Style.valueOf(name());
+    }
+
+    public static Style fromProto(Common.CursorFactory.Style proto) {
+      return Style.valueOf(proto.name());
+    }
   }
 
   /** Result of preparing a statement. */
-  class Signature {
+  public class Signature {
     public final List<ColumnMetaData> columns;
     public final String sql;
     public final List<AvaticaParameter> parameters;
@@ -548,10 +652,118 @@ public interface Meta {
     private <K, V> Map<K, V> sanitize(Map<K, V> map) {
       return map == null ? Collections.<K, V>emptyMap() : map;
     }
+
+    public Common.Signature toProto() {
+      Common.Signature.Builder builder = Common.Signature.newBuilder();
+
+      if (null != sql) {
+        builder.setSql(sql);
+      }
+
+      if (null != cursorFactory) {
+        builder.setCursorFactory(cursorFactory.toProto());
+      }
+
+      if (null != columns) {
+        for (ColumnMetaData column : columns) {
+          builder.addColumns(column.toProto());
+        }
+      }
+
+      if (null != parameters) {
+        for (AvaticaParameter parameter : parameters) {
+          builder.addParameters(parameter.toProto());
+        }
+      }
+
+      return builder.build();
+    }
+
+    public static Signature fromProto(Common.Signature protoSignature) {
+      List<ColumnMetaData> metadata = new ArrayList<>(protoSignature.getColumnsCount());
+      for (Common.ColumnMetaData protoMetadata : protoSignature.getColumnsList()) {
+        metadata.add(ColumnMetaData.fromProto(protoMetadata));
+      }
+
+      List<AvaticaParameter> parameters = new ArrayList<>(protoSignature.getParametersCount());
+      for (Common.AvaticaParameter protoParam : protoSignature.getParametersList()) {
+        parameters.add(AvaticaParameter.fromProto(protoParam));
+      }
+
+      final Descriptor desc = protoSignature.getDescriptorForType();
+
+      String sql = null;
+      if (ProtobufService.hasField(protoSignature, desc, Common.Signature.SQL_FIELD_NUMBER)) {
+        sql = protoSignature.getSql();
+      }
+
+      CursorFactory cursorFactory = null;
+      if (ProtobufService.hasField(protoSignature, desc,
+            Common.Signature.CURSOR_FACTORY_FIELD_NUMBER)) {
+        cursorFactory = CursorFactory.fromProto(protoSignature.getCursorFactory());
+      }
+
+      return Signature.create(metadata, sql, parameters, cursorFactory);
+    }
+
+    @Override public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((columns == null) ? 0 : columns.hashCode());
+      result = prime * result + ((cursorFactory == null) ? 0 : cursorFactory.hashCode());
+      result = prime * result + ((parameters == null) ? 0 : parameters.hashCode());
+      result = prime * result + ((sql == null) ? 0 : sql.hashCode());
+      return result;
+    }
+
+    @Override public boolean equals(Object o) {
+      if (o == this) {
+        return true;
+      }
+      if (o instanceof Signature) {
+        Signature other = (Signature) o;
+
+        if (null == columns) {
+          if (null != other.columns) {
+            return false;
+          }
+        } else if (!columns.equals(other.columns)) {
+          return false;
+        }
+
+        if (null == cursorFactory) {
+          if (null != other.cursorFactory) {
+            return false;
+          }
+        } else if (!cursorFactory.equals(other.cursorFactory)) {
+          return false;
+        }
+
+        if (null == parameters) {
+          if (null != other.parameters) {
+            return false;
+          }
+        } else if (!parameters.equals(other.parameters)) {
+          return false;
+        }
+
+        if (null == sql) {
+          if (null != other.sql) {
+            return false;
+          }
+        } else if (!sql.equals(other.sql)) {
+          return false;
+        }
+
+        return true;
+      }
+
+      return false;
+    }
   }
 
   /** A collection of rows. */
-  class Frame {
+  public class Frame {
     /** Frame that has zero rows and is the last frame. */
     public static final Frame EMPTY =
         new Frame(0, true, Collections.emptyList());
@@ -584,6 +796,212 @@ public interface Meta {
       }
       return new Frame(offset, done, rows);
     }
+
+    public Common.Frame toProto() {
+      Common.Frame.Builder builder = Common.Frame.newBuilder();
+
+      builder.setDone(done).setOffset(offset);
+
+      for (Object row : this.rows) {
+        if (null == row) {
+          // Does this need to be persisted for some reason?
+          continue;
+        }
+
+        if (row instanceof Object[]) {
+          final Common.Row.Builder rowBuilder = Common.Row.newBuilder();
+
+          for (Object element : (Object[]) row) {
+            final Common.TypedValue.Builder valueBuilder = Common.TypedValue.newBuilder();
+
+            // Numbers
+            if (element instanceof Byte) {
+              valueBuilder.setType(Common.Rep.BYTE).setNumberValue(((Byte) element).longValue());
+            } else if (element instanceof Short) {
+              valueBuilder.setType(Common.Rep.SHORT).setNumberValue(((Short) element).longValue());
+            } else if (element instanceof Integer) {
+              valueBuilder.setType(Common.Rep.INTEGER)
+                .setNumberValue(((Integer) element).longValue());
+            } else if (element instanceof Long) {
+              valueBuilder.setType(Common.Rep.LONG).setNumberValue((Long) element);
+            } else if (element instanceof Double) {
+              valueBuilder.setType(Common.Rep.DOUBLE)
+                .setDoubleValue(((Double) element).doubleValue());
+            } else if (element instanceof Float) {
+              valueBuilder.setType(Common.Rep.FLOAT).setNumberValue(((Float) element).longValue());
+            } else if (element instanceof BigDecimal) {
+              valueBuilder.setType(Common.Rep.NUMBER)
+                .setDoubleValue(((BigDecimal) element).doubleValue());
+            // Strings
+            } else if (element instanceof String) {
+              valueBuilder.setType(Common.Rep.STRING)
+                .setStringValue((String) element);
+            } else if (element instanceof Character) {
+              valueBuilder.setType(Common.Rep.CHARACTER)
+                .setStringValue(((Character) element).toString());
+            // Bytes
+            } else if (element instanceof byte[]) {
+              valueBuilder.setType(Common.Rep.BYTE_STRING)
+                .setBytesValues(ByteString.copyFrom((byte[]) element));
+            // Boolean
+            } else if (element instanceof Boolean) {
+              valueBuilder.setType(Common.Rep.BOOLEAN).setBoolValue((boolean) element);
+            } else if (null == element) {
+              valueBuilder.setType(Common.Rep.NULL);
+            // Unhandled
+            } else {
+              throw new RuntimeException("Unhandled type in Frame: " + element.getClass());
+            }
+
+            // Add value to row
+            rowBuilder.addValue(valueBuilder.build());
+          }
+
+          // Collect all rows
+          builder.addRows(rowBuilder.build());
+        } else {
+          // Can a "row" be a primitive? A struct? Only an Array?
+          throw new RuntimeException("Only arrays are supported");
+        }
+      }
+
+      return builder.build();
+    }
+
+    public static Frame fromProto(Common.Frame proto) {
+      List<Object> parsedRows = new ArrayList<>(proto.getRowsCount());
+      for (Common.Row protoRow : proto.getRowsList()) {
+        ArrayList<Object> row = new ArrayList<>(protoRow.getValueCount());
+        for (Common.TypedValue protoElement : protoRow.getValueList()) {
+          Object element;
+
+          // TODO Should these be primitives or Objects?
+          switch (protoElement.getType()) {
+          case BYTE:
+            element = Long.valueOf(protoElement.getNumberValue()).byteValue();
+            break;
+          case SHORT:
+            element = Long.valueOf(protoElement.getNumberValue()).shortValue();
+            break;
+          case INTEGER:
+            element = Long.valueOf(protoElement.getNumberValue()).intValue();
+            break;
+          case LONG:
+            element = protoElement.getNumberValue();
+            break;
+          case FLOAT:
+            element = Long.valueOf(protoElement.getNumberValue()).floatValue();
+            break;
+          case DOUBLE:
+            element = Double.valueOf(protoElement.getDoubleValue());
+            break;
+          case NUMBER:
+            // TODO more cases here to expand on? BigInteger?
+            element = BigDecimal.valueOf(protoElement.getDoubleValue());
+            break;
+          case STRING:
+            element = protoElement.getStringValue();
+            break;
+          case CHARACTER:
+            // A single character in the string
+            element = protoElement.getStringValue().charAt(0);
+            break;
+          case BYTE_STRING:
+            element = protoElement.getBytesValues().toByteArray();
+            break;
+          case BOOLEAN:
+            element = protoElement.getBoolValue();
+            break;
+          case NULL:
+            element = null;
+            break;
+          default:
+            throw new RuntimeException("Unhandled type: " + protoElement.getType());
+          }
+
+          row.add(element);
+        }
+
+        parsedRows.add(row);
+      }
+
+      return new Frame(proto.getOffset(), proto.getDone(), parsedRows);
+    }
+
+    @Override public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + (done ? 1231 : 1237);
+      result = prime * result + (int) (offset ^ (offset >>> 32));
+      result = prime * result + ((rows == null) ? 0 : rows.hashCode());
+      return result;
+    }
+
+    @Override public boolean equals(Object o) {
+      if (o == this) {
+        return true;
+      }
+      if (o instanceof Frame) {
+        Frame other = (Frame) o;
+
+        if (null == rows) {
+          if (null != other.rows) {
+            return false;
+          }
+        } else {
+          Iterator<Object> iter1 = rows.iterator();
+          Iterator<Object> iter2 = other.rows.iterator();
+          while (iter1.hasNext() && iter2.hasNext()) {
+            Object obj1 = iter1.next();
+            Object obj2 = iter2.next();
+
+            // Can't just call equals on an array
+            if (obj1 instanceof Object[]) {
+              if (obj2 instanceof Object[]) {
+                // Compare array and array
+                if (!Arrays.equals((Object[]) obj1, (Object[]) obj2)) {
+                  return false;
+                }
+              } else if (obj2 instanceof List) {
+                // compare array and list
+                @SuppressWarnings("unchecked")
+                List<Object> obj2List = (List<Object>) obj2;
+                if (!Arrays.equals((Object[]) obj1, obj2List.toArray(new Object[0]))) {
+                  return false;
+                }
+              } else {
+                // compare array and something that isn't an array will always fail
+                return false;
+              }
+            } else if (obj1 instanceof List) {
+              if (obj2 instanceof Object[]) {
+                // Compare list and array
+                @SuppressWarnings("unchecked")
+                List<Object> obj1List = (List<Object>) obj1;
+                if (!Arrays.equals(obj1List.toArray(new Object[0]), (Object[]) obj2)) {
+                  return false;
+                }
+              } else if (!obj1.equals(obj2)) {
+                // compare list and something else, let it fall to equals()
+                return false;
+              }
+            } else if (!obj1.equals(obj2)) {
+              // Not an array, leave it to equals()
+              return false;
+            }
+          }
+
+          // More elements in one of the iterables
+          if (iter1.hasNext() || iter2.hasNext()) {
+            return false;
+          }
+        }
+
+        return offset == other.offset && done == other.done;
+      }
+
+      return false;
+    }
   }
 
   /** Connection handle. */
@@ -622,6 +1040,54 @@ public interface Meta {
       this.id = id;
       this.signature = signature;
     }
+
+    public Common.StatementHandle toProto() {
+      return Common.StatementHandle.newBuilder().setConnectionId(connectionId)
+          .setId(id).setSignature(signature.toProto()).build();
+    }
+
+    public static StatementHandle fromProto(Common.StatementHandle protoHandle) {
+      return new StatementHandle(protoHandle.getConnectionId(), protoHandle.getId(),
+          Signature.fromProto(protoHandle.getSignature()));
+    }
+
+    @Override public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((connectionId == null) ? 0 : connectionId.hashCode());
+      result = prime * result + id;
+      result = prime * result + ((signature == null) ? 0 : signature.hashCode());
+      return result;
+    }
+
+    @Override public boolean equals(Object o) {
+      if (o == this) {
+        return true;
+      }
+      if (o instanceof StatementHandle) {
+        StatementHandle other = (StatementHandle) o;
+
+        if (null == connectionId) {
+          if (null != other.connectionId) {
+            return false;
+          }
+        } else if (!connectionId.equals(other.connectionId)) {
+          return false;
+        }
+
+        if (null == signature) {
+          if (null != other.signature) {
+            return false;
+          }
+        } else if (!signature.equals(other.signature)) {
+          return false;
+        }
+
+        return id == other.id;
+      }
+
+      return false;
+    }
   }
 
   /** A pojo containing various client-settable {@link java.sql.Connection} properties.
@@ -691,6 +1157,8 @@ public interface Meta {
     ConnectionProperties setSchema(String val);
 
     String getSchema();
+
+    Common.ConnectionProperties toProto();
   }
 
   /** API to put a result set into a statement, being careful to enforce

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/cb7c213c/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java b/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
index e67b64c..a61fcaf 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
@@ -214,7 +214,7 @@ public abstract class MetaImpl implements Meta {
         Frame.EMPTY);
   }
 
-  protected static ColumnMetaData columnMetaData(String name, int index,
+  public static ColumnMetaData columnMetaData(String name, int index,
       Class<?> type) {
     TypeInfo pair = TypeInfo.m.get(type);
     ColumnMetaData.Rep rep =