From b968e4c39e983fa24e4c1136eafdd1dd45b7cafb Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 13 Oct 2016 17:33:18 +0530 Subject: [PATCH 1/2] APEXMALHAR-1818 Adding BeanClassGenerator for dynamically creating class --- .../sql/codegen/BeanClassGenerator.java | 796 ++++++++++++++++++ .../sql/codegen/BeanClassGeneratorTest.java | 188 +++++ 2 files changed, 984 insertions(+) create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/codegen/BeanClassGenerator.java create mode 100644 sql/src/test/java/org/apache/apex/malhar/sql/codegen/BeanClassGeneratorTest.java diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/codegen/BeanClassGenerator.java b/sql/src/main/java/org/apache/apex/malhar/sql/codegen/BeanClassGenerator.java new file mode 100644 index 0000000000..ffeafb509d --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/codegen/BeanClassGenerator.java @@ -0,0 +1,796 @@ +/** + * 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.apex.malhar.sql.codegen; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import org.codehaus.jettison.json.JSONException; + +import org.apache.apex.malhar.sql.schema.TupleSchemaRegistry; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.xbean.asm5.ClassWriter; +import org.apache.xbean.asm5.Opcodes; +import org.apache.xbean.asm5.tree.ClassNode; +import org.apache.xbean.asm5.tree.FieldInsnNode; +import org.apache.xbean.asm5.tree.FieldNode; +import org.apache.xbean.asm5.tree.InsnNode; +import org.apache.xbean.asm5.tree.IntInsnNode; +import org.apache.xbean.asm5.tree.JumpInsnNode; +import org.apache.xbean.asm5.tree.LabelNode; +import org.apache.xbean.asm5.tree.LdcInsnNode; +import org.apache.xbean.asm5.tree.MethodInsnNode; +import org.apache.xbean.asm5.tree.MethodNode; +import org.apache.xbean.asm5.tree.TypeInsnNode; +import org.apache.xbean.asm5.tree.VarInsnNode; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; + +/** + * Creates a bean class on fly. + */ +@InterfaceStability.Evolving +public class BeanClassGenerator +{ + public static final ImmutableMap PRIMITIVE_TYPES; + + static { + Map types = Maps.newHashMap(); + types.put("boolean", 'Z'); + types.put("char", 'C'); + types.put("byte", 'B'); + types.put("short", 'S'); + types.put("int", 'I'); + types.put("float", 'F'); + types.put("long", 'J'); + types.put("double", 'D'); + PRIMITIVE_TYPES = ImmutableMap.copyOf(types); + } + + /** + * Creates a class from give field information and returns byte array of compiled class. + * + * @param fqcn fully qualified class name + * @param fieldList field list for which POJO needs to be generated. + * + * @return byte[] representing compiled class. + * @throws IOException + * @throws JSONException + */ + public static byte[] createAndWriteBeanClass(String fqcn, List fieldList) + throws IOException, JSONException + { + return createAndWriteBeanClass(fqcn, fieldList, null); + } + + /** + * Creates a class from given field information and writes it to the output stream. Also returns byte[] of compiled + * class + * + * @param fqcn fully qualified class name + * @param fieldList field list describing the class + * @param outputStream stream to which the class is persisted + * @throws JSONException + * @throws IOException + */ + @SuppressWarnings("unchecked") + public static byte[] createAndWriteBeanClass(String fqcn, List fieldList, + FSDataOutputStream outputStream) throws JSONException, IOException + { + ClassNode classNode = new ClassNode(); + + classNode.version = Opcodes.V1_6; //generated class will only run on JRE 1.6 or above + classNode.access = Opcodes.ACC_PUBLIC; + + classNode.name = fqcn.replace('.', '/'); + classNode.superName = "java/lang/Object"; + + // add default constructor + addDefaultConstructor(classNode); + + //for each field in json add a field to this class and a getter and setter for it. + + for (TupleSchemaRegistry.SQLFieldInfo fieldInfo : fieldList) { + String fieldName = fieldInfo.getColumnName(); + String fieldType = fieldInfo.getType().getJavaType().getName(); + String fieldJavaType = getJavaType(fieldType); + + // Add private field + FieldNode fieldNode = new FieldNode(Opcodes.ACC_PRIVATE, fieldName, fieldJavaType, null, null); + classNode.fields.add(fieldNode); + + String fieldNameForMethods = Character.toUpperCase(fieldName.charAt(0)) + fieldName.substring(1); + + switch (fieldType) { + case "boolean": + addIntGetterNSetter(classNode, fieldName, fieldNameForMethods, fieldJavaType, true); + break; + case "byte": + case "char": + case "short": + case "int": + addIntGetterNSetter(classNode, fieldName, fieldNameForMethods, fieldJavaType, false); + break; + case "long": + addLongGetterNSetter(classNode, fieldName, fieldNameForMethods, fieldJavaType); + break; + case "float": + addFloatGetterNSetter(classNode, fieldName, fieldNameForMethods, fieldJavaType); + break; + case "double": + addDoubleGetterNSetter(classNode, fieldName, fieldNameForMethods, fieldJavaType); + break; + default: + if (fieldJavaType.equals(getJavaType("java.util.Date"))) { + addDateFields(classNode, fieldName, fieldNameForMethods, "java/util/Date"); + } else { + addObjectGetterNSetter(classNode, fieldName, fieldNameForMethods, fieldJavaType); + } + break; + } + } + + addToStringMethod(classNode, fieldList); + addHashCodeMethod(classNode, fieldList); + addEqualsMethod(classNode, fieldList); + + //Write the class + ClassWriter cw = new ClassWriter(ClassWriter.COMPUTE_MAXS | ClassWriter.COMPUTE_FRAMES); + classNode.accept(cw); + cw.visitEnd(); + + byte[] classBytes = cw.toByteArray(); + + if (outputStream != null) { + outputStream.write(classBytes); + outputStream.close(); + } + + return classBytes; + } + + private static void addDefaultConstructor(ClassNode classNode) + { + MethodNode constructorNode = new MethodNode(Opcodes.ACC_PUBLIC, "", "()V", null, null); + constructorNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + constructorNode.instructions + .add(new MethodInsnNode(Opcodes.INVOKESPECIAL, "java/lang/Object", "", "()V", false)); + constructorNode.instructions.add(new InsnNode(Opcodes.RETURN)); + classNode.methods.add(constructorNode); + } + + /** + * Date field is explicitly handled and provided with 3 variants of types of same data. + * 1. java.util.Date format + * 2. long - Epoc time in ms + * 3. int - Epoc time in sec rounded to date + * + * This is purposefully done because SQL operations on Date etc happens on long or int based on whether its a SQL DATE + * field OR SQL TIMESTAMP field. Hence to cater to that 2 more variant of the same data is added to the POJO. + */ + @SuppressWarnings("unchecked") + private static void addDateFields(ClassNode classNode, String fieldName, String fieldNameForMethods, String type) + { + FieldNode fieldNodeSec = new FieldNode(Opcodes.ACC_PRIVATE, fieldName + "Sec", getJavaType("java.lang.Integer"), + null, null); + classNode.fields.add(fieldNodeSec); + FieldNode fieldNodeMs = new FieldNode(Opcodes.ACC_PRIVATE, fieldName + "Ms", getJavaType("java.lang.Long"), null, + null); + classNode.fields.add(fieldNodeMs); + + // Create getter for Date + MethodNode getterNodeDate = new MethodNode(Opcodes.ACC_PUBLIC, "get" + fieldNameForMethods, "()L" + type + ";", + null, null); + getterNodeDate.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + getterNodeDate.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, "L" + type + ";")); + getterNodeDate.instructions.add(new InsnNode(Opcodes.ARETURN)); + classNode.methods.add(getterNodeDate); + + // Create getter for Sec + MethodNode getterNodeSec = new MethodNode(Opcodes.ACC_PUBLIC, "get" + fieldNameForMethods + "Sec", + "()Ljava/lang/Integer;", null, null); + getterNodeSec.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + getterNodeSec.instructions + .add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName + "Sec", "Ljava/lang/Integer;")); + getterNodeSec.instructions.add(new InsnNode(Opcodes.ARETURN)); + classNode.methods.add(getterNodeSec); + + // Create getter for Ms + MethodNode getterNodeMs = new MethodNode(Opcodes.ACC_PUBLIC, "get" + fieldNameForMethods + "Ms", + "()Ljava/lang/Long;", null, null); + getterNodeMs.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + getterNodeMs.instructions + .add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName + "Ms", "Ljava/lang/Long;")); + getterNodeMs.instructions.add(new InsnNode(Opcodes.ARETURN)); + classNode.methods.add(getterNodeMs); + + // Create setter for Date + MethodNode setterNodeDate = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods, + "(L" + type + ";)V", null, null); + setterNodeDate.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNodeDate.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + setterNodeDate.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, "L" + type + ";")); + + setterNodeDate.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNodeDate.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + setterNodeDate.instructions.add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, type, "getTime", "()J", false)); + setterNodeDate.instructions.add(new LdcInsnNode(new Long(1000))); + setterNodeDate.instructions.add(new InsnNode(Opcodes.LDIV)); + setterNodeDate.instructions.add(new InsnNode(Opcodes.L2I)); + setterNodeDate.instructions + .add(new MethodInsnNode(Opcodes.INVOKESTATIC, "java/lang/Integer", "valueOf", "(I)Ljava/lang/Integer;", false)); + setterNodeDate.instructions + .add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName + "Sec", "Ljava/lang/Integer;")); + + setterNodeDate.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNodeDate.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + setterNodeDate.instructions.add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, type, "getTime", "()J", false)); + setterNodeDate.instructions + .add(new MethodInsnNode(Opcodes.INVOKESTATIC, "java/lang/Long", "valueOf", "(J)Ljava/lang/Long;", false)); + setterNodeDate.instructions + .add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName + "Ms", "Ljava/lang/Long;")); + + setterNodeDate.instructions.add(new InsnNode(Opcodes.RETURN)); + classNode.methods.add(setterNodeDate); + + // Create setter for Sec + MethodNode setterNodeSec = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods + "Sec", + "(Ljava/lang/Integer;)V", null, null); + setterNodeSec.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNodeSec.instructions.add(new TypeInsnNode(Opcodes.NEW, type)); + setterNodeSec.instructions.add(new InsnNode(Opcodes.DUP)); + setterNodeSec.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + setterNodeSec.instructions + .add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, "java/lang/Integer", "intValue", "()I", false)); + setterNodeSec.instructions.add(new InsnNode(Opcodes.I2L)); + setterNodeSec.instructions.add(new LdcInsnNode(new Long(1000))); + setterNodeSec.instructions.add(new InsnNode(Opcodes.LMUL)); + setterNodeSec.instructions + .add(new MethodInsnNode(Opcodes.INVOKESPECIAL, "java/util/Date", "", "(J)V", false)); + setterNodeSec.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, "L" + type + ";")); + + setterNodeSec.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNodeSec.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + setterNodeSec.instructions + .add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName + "Sec", "Ljava/lang/Integer;")); + + setterNodeSec.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNodeSec.instructions.add(new TypeInsnNode(Opcodes.NEW, "java/lang/Long")); + setterNodeSec.instructions.add(new InsnNode(Opcodes.DUP)); + setterNodeSec.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + setterNodeSec.instructions + .add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, "java/lang/Integer", "intValue", "()I", false)); + setterNodeSec.instructions.add(new InsnNode(Opcodes.I2L)); + setterNodeSec.instructions.add(new LdcInsnNode(new Long(1000))); + setterNodeSec.instructions.add(new InsnNode(Opcodes.LMUL)); + setterNodeSec.instructions + .add(new MethodInsnNode(Opcodes.INVOKESPECIAL, "java/lang/Long", "", "(J)V", false)); + setterNodeSec.instructions + .add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName + "Ms", "Ljava/lang/Long;")); + + setterNodeSec.instructions.add(new InsnNode(Opcodes.RETURN)); + classNode.methods.add(setterNodeSec); + + // Create setter for Ms + MethodNode setterNodeMs = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods + "Ms", + "(Ljava/lang/Long;)V", null, null); + setterNodeMs.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNodeMs.instructions.add(new TypeInsnNode(Opcodes.NEW, type)); + setterNodeMs.instructions.add(new InsnNode(Opcodes.DUP)); + setterNodeMs.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + setterNodeMs.instructions + .add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, "java/lang/Long", "longValue", "()J", false)); + setterNodeMs.instructions.add(new MethodInsnNode(Opcodes.INVOKESPECIAL, "java/util/Date", "", "(J)V", false)); + setterNodeMs.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, "L" + type + ";")); + + setterNodeMs.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNodeMs.instructions.add(new TypeInsnNode(Opcodes.NEW, "java/lang/Integer")); + setterNodeMs.instructions.add(new InsnNode(Opcodes.DUP)); + setterNodeMs.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + setterNodeMs.instructions + .add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, "java/lang/Long", "longValue", "()J", false)); + setterNodeMs.instructions.add(new LdcInsnNode(new Long(1000))); + setterNodeMs.instructions.add(new InsnNode(Opcodes.LDIV)); + setterNodeMs.instructions.add(new InsnNode(Opcodes.L2I)); + setterNodeMs.instructions + .add(new MethodInsnNode(Opcodes.INVOKESPECIAL, "java/lang/Integer", "", "(I)V", false)); + setterNodeMs.instructions + .add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName + "Sec", "Ljava/lang/Integer;")); + + setterNodeMs.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNodeMs.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + setterNodeMs.instructions + .add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName + "Ms", "Ljava/lang/Long;")); + + setterNodeMs.instructions.add(new InsnNode(Opcodes.RETURN)); + classNode.methods.add(setterNodeMs); + } + + @SuppressWarnings("unchecked") + private static void addIntGetterNSetter(ClassNode classNode, String fieldName, String fieldNameForMethods, + String fieldJavaType, boolean isBoolean) + { + // Create getter + String getterSignature = "()" + fieldJavaType; + MethodNode getterNode = new MethodNode(Opcodes.ACC_PUBLIC, (isBoolean ? "is" : "get") + fieldNameForMethods, + getterSignature, null, null); + getterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + getterNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); + getterNode.instructions.add(new InsnNode(Opcodes.IRETURN)); + classNode.methods.add(getterNode); + + // Create setter + String setterSignature = '(' + fieldJavaType + ')' + 'V'; + MethodNode setterNode = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods, setterSignature, null, + null); + setterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNode.instructions.add(new VarInsnNode(Opcodes.ILOAD, 1)); + setterNode.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, fieldJavaType)); + setterNode.instructions.add(new InsnNode(Opcodes.RETURN)); + classNode.methods.add(setterNode); + } + + @SuppressWarnings("unchecked") + private static void addLongGetterNSetter(ClassNode classNode, String fieldName, String fieldNameForMethods, + String fieldJavaType) + { + // Create getter + String getterSignature = "()" + fieldJavaType; + MethodNode getterNode = new MethodNode(Opcodes.ACC_PUBLIC, "get" + fieldNameForMethods, getterSignature, null, + null); + getterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + getterNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); + getterNode.instructions.add(new InsnNode(Opcodes.LRETURN)); + classNode.methods.add(getterNode); + + // Create setter + String setterSignature = '(' + fieldJavaType + ')' + 'V'; + MethodNode setterNode = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods, setterSignature, null, + null); + setterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNode.instructions.add(new VarInsnNode(Opcodes.LLOAD, 1)); + setterNode.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, fieldJavaType)); + setterNode.instructions.add(new InsnNode(Opcodes.RETURN)); + classNode.methods.add(setterNode); + } + + @SuppressWarnings("unchecked") + private static void addFloatGetterNSetter(ClassNode classNode, String fieldName, String fieldNameForMethods, + String fieldJavaType) + { + // Create getter + String getterSignature = "()" + fieldJavaType; + MethodNode getterNode = new MethodNode(Opcodes.ACC_PUBLIC, "get" + fieldNameForMethods, getterSignature, null, + null); + getterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + getterNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); + getterNode.instructions.add(new InsnNode(Opcodes.FRETURN)); + classNode.methods.add(getterNode); + + // Create setter + String setterSignature = '(' + fieldJavaType + ')' + 'V'; + MethodNode setterNode = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods, setterSignature, null, + null); + setterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNode.instructions.add(new VarInsnNode(Opcodes.FLOAD, 1)); + setterNode.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, fieldJavaType)); + setterNode.instructions.add(new InsnNode(Opcodes.RETURN)); + classNode.methods.add(setterNode); + } + + @SuppressWarnings("unchecked") + private static void addDoubleGetterNSetter(ClassNode classNode, String fieldName, String fieldNameForMethods, + String fieldJavaType) + { + // Create getter + String getterSignature = "()" + fieldJavaType; + MethodNode getterNode = new MethodNode(Opcodes.ACC_PUBLIC, "get" + fieldNameForMethods, getterSignature, null, + null); + getterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + getterNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); + getterNode.instructions.add(new InsnNode(Opcodes.DRETURN)); + classNode.methods.add(getterNode); + + // Create setter + String setterSignature = '(' + fieldJavaType + ')' + 'V'; + MethodNode setterNode = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods, setterSignature, null, + null); + setterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNode.instructions.add(new VarInsnNode(Opcodes.DLOAD, 1)); + setterNode.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, fieldJavaType)); + setterNode.instructions.add(new InsnNode(Opcodes.RETURN)); + classNode.methods.add(setterNode); + } + + @SuppressWarnings("unchecked") + private static void addObjectGetterNSetter(ClassNode classNode, String fieldName, String fieldNameForMethods, + String fieldJavaType) + { + // Create getter + String getterSignature = "()" + fieldJavaType; + MethodNode getterNode = new MethodNode(Opcodes.ACC_PUBLIC, "get" + fieldNameForMethods, getterSignature, null, + null); + getterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + getterNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); + getterNode.instructions.add(new InsnNode(Opcodes.ARETURN)); + classNode.methods.add(getterNode); + + // Create setter + String setterSignature = '(' + fieldJavaType + ')' + 'V'; + MethodNode setterNode = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods, setterSignature, null, + null); + setterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + setterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + setterNode.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, fieldJavaType)); + setterNode.instructions.add(new InsnNode(Opcodes.RETURN)); + classNode.methods.add(setterNode); + } + + /** + * Adds a toString method to underlying class. Uses StringBuilder to generate the final string. + * + * @param classNode + * @param fieldList + * @throws JSONException + */ + @SuppressWarnings("unchecked") + private static void addToStringMethod(ClassNode classNode, List fieldList) + throws JSONException + { + MethodNode toStringNode = new MethodNode(Opcodes.ACC_PUBLIC, "toString", "()Ljava/lang/String;", null, null); + toStringNode.visitAnnotation("Ljava/lang/Override;", true); + + toStringNode.instructions.add(new TypeInsnNode(Opcodes.NEW, "java/lang/StringBuilder")); + toStringNode.instructions.add(new InsnNode(Opcodes.DUP)); + toStringNode.instructions.add(new LdcInsnNode(classNode.name + "{")); + toStringNode.instructions.add(new MethodInsnNode(Opcodes.INVOKESPECIAL, "java/lang/StringBuilder", + "", "(Ljava/lang/String;)V", false)); + toStringNode.instructions.add(new VarInsnNode(Opcodes.ASTORE, 1)); + toStringNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + + for (int i = 0; i < fieldList.size(); i++) { + TupleSchemaRegistry.SQLFieldInfo info = fieldList.get(i); + String fieldName = info.getColumnName(); + String fieldType = info.getType().getJavaType().getName(); + String fieldJavaType = getJavaType(fieldType); + + if (i != 0) { + toStringNode.instructions.add(new LdcInsnNode(", ")); + toStringNode.instructions.add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, "java/lang/StringBuilder", "append", + "(Ljava/lang/String;)Ljava/lang/StringBuilder;", false)); + } + + toStringNode.instructions.add(new LdcInsnNode(fieldName + "=")); + toStringNode.instructions.add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, "java/lang/StringBuilder", "append", + "(Ljava/lang/String;)Ljava/lang/StringBuilder;", false)); + toStringNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + toStringNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); + + // There is no StringBuilder.append method for short and byte. It takes it as int. + if (fieldJavaType.equals("S") || fieldJavaType.equals("B")) { + fieldJavaType = "I"; + } + + Character pchar = PRIMITIVE_TYPES.get(fieldType); + if (pchar == null) { + // It's not a primitive type. StringBuilder.append method signature takes Object type. + fieldJavaType = "Ljava/lang/Object;"; + } + + toStringNode.instructions.add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, "java/lang/StringBuilder", "append", + "(" + fieldJavaType + ")Ljava/lang/StringBuilder;", false)); + } + + toStringNode.instructions.add(new LdcInsnNode("}")); + toStringNode.instructions.add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, "java/lang/StringBuilder", "append", + "(Ljava/lang/String;)Ljava/lang/StringBuilder;", false)); + + toStringNode.instructions.add(new InsnNode(Opcodes.POP)); + toStringNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + toStringNode.instructions.add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, "java/lang/StringBuilder", "toString", + "()Ljava/lang/String;", false)); + toStringNode.instructions.add(new InsnNode(Opcodes.ARETURN)); + + classNode.methods.add(toStringNode); + } + + /** + * This will add a hashCode method for class being generated.
+ * Algorithm is as follows:
+ *

+ * int hashCode = 7; + * for (field: all fields) { + * hashCode = 23 * hashCode + field.hashCode() + * } + *

+ *
+ * For primitive field, hashcode implemenented is similar to the one present in its wrapper class. + * + * @param classNode + * @param fieldList + * @throws JSONException + */ + @SuppressWarnings("unchecked") + private static void addHashCodeMethod(ClassNode classNode, List fieldList) + throws JSONException + { + MethodNode hashCodeNode = new MethodNode(Opcodes.ACC_PUBLIC, "hashCode", "()I", null, null); + hashCodeNode.visitAnnotation("Ljava/lang/Override;", true); + + hashCodeNode.instructions.add(new IntInsnNode(Opcodes.BIPUSH, 7)); + hashCodeNode.instructions.add(new VarInsnNode(Opcodes.ISTORE, 1)); + + for (TupleSchemaRegistry.SQLFieldInfo fieldInfo : fieldList) { + String fieldName = fieldInfo.getColumnName(); + String fieldType = fieldInfo.getType().getJavaType().getName(); + String fieldJavaType = getJavaType(fieldType); + + hashCodeNode.instructions.add(new IntInsnNode(Opcodes.BIPUSH, 23)); + hashCodeNode.instructions.add(new VarInsnNode(Opcodes.ILOAD, 1)); + hashCodeNode.instructions.add(new InsnNode(Opcodes.IMUL)); + hashCodeNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + hashCodeNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); + + switch (fieldType) { + case "boolean": + LabelNode falseNode = new LabelNode(); + LabelNode trueNode = new LabelNode(); + hashCodeNode.instructions.add(new JumpInsnNode(Opcodes.IFEQ, falseNode)); + hashCodeNode.instructions.add(new IntInsnNode(Opcodes.SIPUSH, 1231)); + hashCodeNode.instructions.add(new JumpInsnNode(Opcodes.GOTO, trueNode)); + hashCodeNode.instructions.add(falseNode); + hashCodeNode.instructions.add(new IntInsnNode(Opcodes.SIPUSH, 1237)); + hashCodeNode.instructions.add(trueNode); + break; + case "byte": + case "char": + case "short": + case "int": + break; + case "float": + hashCodeNode.instructions + .add(new MethodInsnNode(Opcodes.INVOKESTATIC, "java/lang/Float", "floatToIntBits", "(F)I", false)); + break; + case "long": + hashCodeNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + hashCodeNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); + hashCodeNode.instructions.add(new IntInsnNode(Opcodes.BIPUSH, 32)); + hashCodeNode.instructions.add(new InsnNode(Opcodes.LUSHR)); + hashCodeNode.instructions.add(new InsnNode(Opcodes.LXOR)); + hashCodeNode.instructions.add(new InsnNode(Opcodes.L2I)); + break; + case "double": + hashCodeNode.instructions + .add(new MethodInsnNode(Opcodes.INVOKESTATIC, "java/lang/Double", "doubleToLongBits", "(D)J", false)); + hashCodeNode.instructions.add(new InsnNode(Opcodes.DUP2)); + hashCodeNode.instructions.add(new VarInsnNode(Opcodes.LSTORE, 2)); + hashCodeNode.instructions.add(new VarInsnNode(Opcodes.LLOAD, 2)); + hashCodeNode.instructions.add(new IntInsnNode(Opcodes.BIPUSH, 32)); + hashCodeNode.instructions.add(new InsnNode(Opcodes.LUSHR)); + hashCodeNode.instructions.add(new InsnNode(Opcodes.LXOR)); + hashCodeNode.instructions.add(new InsnNode(Opcodes.L2I)); + break; + default: + String objectOwnerType = fieldType.replace('.', '/'); + LabelNode nullNode = new LabelNode(); + LabelNode continueNode = new LabelNode(); + hashCodeNode.instructions.add(new JumpInsnNode(Opcodes.IFNULL, nullNode)); + hashCodeNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + hashCodeNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); + hashCodeNode.instructions + .add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, objectOwnerType, "hashCode", "()I", false)); + hashCodeNode.instructions.add(new JumpInsnNode(Opcodes.GOTO, continueNode)); + hashCodeNode.instructions.add(nullNode); + hashCodeNode.instructions.add(new InsnNode(Opcodes.ICONST_0)); + hashCodeNode.instructions.add(continueNode); + break; + } + hashCodeNode.instructions.add(new InsnNode(Opcodes.IADD)); + hashCodeNode.instructions.add(new VarInsnNode(Opcodes.ISTORE, 1)); + } + hashCodeNode.instructions.add(new VarInsnNode(Opcodes.ILOAD, 1)); + hashCodeNode.instructions.add(new InsnNode(Opcodes.IRETURN)); + + classNode.methods.add(hashCodeNode); + } + + /** + * Adds a equals method to underlying class.
+ * Algorithm is as follows:
+ *

+ * if (this == other) return true; + * if (other == null) return false; + * if (other is not instanceof ) return false; + * for (field: all fields) { + * if (other.getField() != this.field) return false; + * } + * return true; + *

+ *
+ * + * @param classNode + * @param fieldList + * @throws JSONException + */ + @SuppressWarnings("unchecked") + private static void addEqualsMethod(ClassNode classNode, List fieldList) + throws JSONException + { + MethodNode equalsNode = new MethodNode(Opcodes.ACC_PUBLIC, "equals", "(Ljava/lang/Object;)Z", null, null); + equalsNode.visitAnnotation("Ljava/lang/Override;", true); + + LabelNode l0 = new LabelNode(); + LabelNode l1 = new LabelNode(); + LabelNode l2 = new LabelNode(); + LabelNode l3 = new LabelNode(); + LabelNode l4 = new LabelNode(); + + equalsNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + + // if (this == other) return true; + equalsNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + equalsNode.instructions.add(new JumpInsnNode(Opcodes.IF_ACMPNE, l0)); + equalsNode.instructions.add(new InsnNode(Opcodes.ICONST_1)); + equalsNode.instructions.add(new InsnNode(Opcodes.IRETURN)); + + equalsNode.instructions.add(l0); + // if (other == null) return false; + equalsNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + equalsNode.instructions.add(new JumpInsnNode(Opcodes.IFNULL, l1)); + // if (!(other instanceof )) return false; + equalsNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + equalsNode.instructions.add(new TypeInsnNode(Opcodes.INSTANCEOF, classNode.name)); + equalsNode.instructions.add(new JumpInsnNode(Opcodes.IFNE, l2)); + + equalsNode.instructions.add(l1); + equalsNode.instructions.add(new InsnNode(Opcodes.ICONST_0)); + equalsNode.instructions.add(new InsnNode(Opcodes.IRETURN)); + + equalsNode.instructions.add(l2); + // Check if it other object can cast to + equalsNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); + equalsNode.instructions.add(new TypeInsnNode(Opcodes.CHECKCAST, classNode.name)); + equalsNode.instructions.add(new VarInsnNode(Opcodes.ASTORE, 2)); + + for (int i = 0; i < fieldList.size(); i++) { + boolean isLast = ((i + 1) == fieldList.size()); + TupleSchemaRegistry.SQLFieldInfo info = fieldList.get(i); + String fieldName = info.getColumnName(); + String fieldType = info.getType().getJavaType().getName(); + String fieldJavaType = getJavaType(fieldType); + + String getterMethodName = (fieldType.equals("boolean") ? "is" : "get") + + Character.toUpperCase(fieldName.charAt(0)) + fieldName.substring(1); + equalsNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 2)); + equalsNode.instructions.add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, classNode.name, getterMethodName, + "()" + fieldJavaType, false)); + equalsNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + equalsNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); + + switch (fieldType) { + case "boolean": + case "byte": + case "char": + case "short": + case "int": + equalsNode.instructions + .add(new JumpInsnNode(isLast ? Opcodes.IF_ICMPEQ : Opcodes.IF_ICMPNE, isLast ? l4 : l3)); + break; + case "long": + equalsNode.instructions.add(new InsnNode(Opcodes.LCMP)); + equalsNode.instructions.add(new JumpInsnNode(isLast ? Opcodes.IFEQ : Opcodes.IFNE, isLast ? l4 : l3)); + break; + case "float": + equalsNode.instructions.add(new InsnNode(Opcodes.FCMPL)); + equalsNode.instructions.add(new JumpInsnNode(isLast ? Opcodes.IFEQ : Opcodes.IFNE, isLast ? l4 : l3)); + break; + case "double": + equalsNode.instructions.add(new InsnNode(Opcodes.DCMPL)); + equalsNode.instructions.add(new JumpInsnNode(isLast ? Opcodes.IFEQ : Opcodes.IFNE, isLast ? l4 : l3)); + break; + default: + String objectOwnerType = fieldType.replace('.', '/'); + + LabelNode nonNullNode = new LabelNode(); + LabelNode continueNode = new LabelNode(); + + equalsNode.instructions.add(new JumpInsnNode(Opcodes.IFNONNULL, nonNullNode)); + equalsNode.instructions.add(new JumpInsnNode(isLast ? Opcodes.IFNULL : Opcodes.IFNONNULL, isLast ? l4 : l3)); + + equalsNode.instructions.add(new JumpInsnNode(Opcodes.GOTO, continueNode)); + + equalsNode.instructions.add(nonNullNode); + equalsNode.instructions.add(new InsnNode(Opcodes.POP)); + equalsNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + equalsNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); + equalsNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 2)); + equalsNode.instructions.add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, classNode.name, getterMethodName, + "()" + fieldJavaType, false)); + equalsNode.instructions.add(new MethodInsnNode(Opcodes.INVOKEVIRTUAL, objectOwnerType, "equals", + "(Ljava/lang/Object;)Z", false)); + equalsNode.instructions.add(new JumpInsnNode(isLast ? Opcodes.IFNE : Opcodes.IFEQ, isLast ? l4 : l3)); + + equalsNode.instructions.add(continueNode); + break; + } + } + + equalsNode.instructions.add(l3); + equalsNode.instructions.add(new InsnNode(Opcodes.ICONST_0)); + equalsNode.instructions.add(new InsnNode(Opcodes.IRETURN)); + + equalsNode.instructions.add(l4); + equalsNode.instructions.add(new InsnNode(Opcodes.ICONST_1)); + equalsNode.instructions.add(new InsnNode(Opcodes.IRETURN)); + + classNode.methods.add(equalsNode); + } + + private static String getJavaType(String fieldType) + { + Character pchar = PRIMITIVE_TYPES.get(fieldType); + if (pchar != null) { + //it is a primitive type + return Character.toString(pchar); + } + //non-primitive so find the internal name of the class. + return 'L' + fieldType.replace('.', '/') + ';'; + } + + /** + * Given the class name it reads and loads the class from the input stream. + * + * @param fqcn fully qualified class name. + * @param inputStream stream from which class is read. + * @return loaded class + * @throws IOException + */ + public static Class readBeanClass(String fqcn, FSDataInputStream inputStream) throws IOException + { + byte[] bytes = IOUtils.toByteArray(inputStream); + inputStream.close(); + return new ByteArrayClassLoader().defineClass(fqcn, bytes); + } + + /** + * Given the class name it reads and loads the class from given byte array. + * + * @param fqcn fully qualified class name. + * @param inputClass byte[] from which class is read. + * @return loaded class + * @throws IOException + */ + public static Class readBeanClass(String fqcn, byte[] inputClass) throws IOException + { + return new ByteArrayClassLoader().defineClass(fqcn, inputClass); + } + + private static class ByteArrayClassLoader extends ClassLoader + { + Class defineClass(String name, byte[] ba) + { + return defineClass(name, ba, 0, ba.length); + } + } +} diff --git a/sql/src/test/java/org/apache/apex/malhar/sql/codegen/BeanClassGeneratorTest.java b/sql/src/test/java/org/apache/apex/malhar/sql/codegen/BeanClassGeneratorTest.java new file mode 100644 index 0000000000..8fcb7f8692 --- /dev/null +++ b/sql/src/test/java/org/apache/apex/malhar/sql/codegen/BeanClassGeneratorTest.java @@ -0,0 +1,188 @@ +/** + * Copyright (c) 2015 DataTorrent, Inc. + * All rights reserved. + */ +package org.apache.apex.malhar.sql.codegen; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Date; + +import org.codehaus.jettison.json.JSONException; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestWatcher; +import org.junit.runner.Description; + +import org.apache.apex.malhar.sql.schema.TupleSchemaRegistry; + +import static org.junit.Assert.assertEquals; + +public class BeanClassGeneratorTest +{ + + protected class TestMeta extends TestWatcher + { + String generatedDir; + + @Override + protected void starting(Description description) + { + super.starting(description); + generatedDir = "target/" + description.getClassName() + "/" + description.getMethodName(); + } + + @Override + protected void finished(Description description) + { + super.finished(description); + } + } + + @Rule + public TestMeta testMeta = new TestMeta(); + + @Test + public void test() throws IOException, JSONException, IllegalAccessException, InstantiationException, + NoSuchFieldException, NoSuchMethodException, InvocationTargetException + { + String addressClassName = TupleSchemaRegistry.FQCN_PACKAGE + "Address_v1"; + + TupleSchemaRegistry.Schema schema = new TupleSchemaRegistry.Schema(); + schema.addField("streetNumber", long.class); + schema.addField("streetName", String.class); + schema.addField("city", String.class); + schema.addField("state", String.class); + schema.addField("zip", String.class); + + byte[] beanClass = BeanClassGenerator.createAndWriteBeanClass(addressClassName, schema.fieldList); + + Class clazz = BeanClassGenerator.readBeanClass(addressClassName, beanClass); + + Object o = clazz.newInstance(); + Field f = clazz.getDeclaredField("streetNumber"); + Assert.assertNotNull(f); + + Method m = clazz.getDeclaredMethod("setStreetNumber", Long.class); + m.invoke(o, 343L); + + m = clazz.getMethod("getStreetNumber"); + Long result = (Long)m.invoke(o); + + assertEquals("reflect getVal invoke", 343, result.longValue()); + } + + @Test + public void testPrimitive() throws IOException, JSONException, IllegalAccessException, InstantiationException, + NoSuchFieldException, NoSuchMethodException, InvocationTargetException + { + String addressClassName = TupleSchemaRegistry.FQCN_PACKAGE + "Energy_v1"; + + TupleSchemaRegistry.Schema schema = new TupleSchemaRegistry.Schema(); + schema.addField("streetNumber", Integer.class); + schema.addField("houseNumber", Long.class); + schema.addField("condo", Boolean.class); + schema.addField("water-usage", Float.class); + schema.addField("electricity-usage", Double.class); + schema.addField("startDate", Date.class); + + byte[] beanClass = BeanClassGenerator.createAndWriteBeanClass(addressClassName, schema.fieldList); + + Class clazz = BeanClassGenerator.readBeanClass(addressClassName, beanClass); + + Object o = clazz.newInstance(); + Field f = clazz.getDeclaredField("streetNumber"); + Assert.assertNotNull(f); + + //int setter and getter + Method m = clazz.getDeclaredMethod("setStreetNumber", Integer.class); + m.invoke(o, 343); + m = clazz.getMethod("getStreetNumber"); + Integer result = (Integer)m.invoke(o); + + assertEquals("reflect getStreetNumber invoke", 343, result.intValue()); + + //long setter and getter + m = clazz.getDeclaredMethod("setHouseNumber", Long.class); + m.invoke(o, 123L); + m = clazz.getMethod("getHouseNumber"); + Long houseNum = (Long)m.invoke(o); + + assertEquals("reflect getHouseNumber invoke", 123L, houseNum.longValue()); + + //boolean setter and getter + m = clazz.getDeclaredMethod("setCondo", Boolean.class); + m.invoke(o, true); + m = clazz.getMethod("getCondo"); + Boolean isCondo = (Boolean)m.invoke(o); + + assertEquals("reflect getCondo invoke", true, isCondo); + + //float setter and getter + m = clazz.getDeclaredMethod("setWater-usage", Float.class); + m.invoke(o, 88.34F); + m = clazz.getMethod("getWater-usage"); + Float waterUsage = (Float)m.invoke(o); + + assertEquals("reflect getWaterUsage invoke", 88.34F, waterUsage.floatValue(), 0); + + //double setter and getter + m = clazz.getDeclaredMethod("setElectricity-usage", Double.class); + m.invoke(o, 88.343243); + m = clazz.getMethod("getElectricity-usage"); + Double electricityUsage = (Double)m.invoke(o); + + assertEquals("reflect getWaterUsage invoke", 88.343243, electricityUsage, 0); + + Date now = new Date(); + m = clazz.getDeclaredMethod("setStartDate", Date.class); + m.invoke(o, now); + + m = clazz.getMethod("getStartDate"); + Date startDate = (Date)m.invoke(o); + assertEquals("reflect getStartDate invoke", now, startDate); + + m = clazz.getMethod("getStartDateMs"); + long startDateMs = (long)m.invoke(o); + assertEquals("reflect getStartDateMs invoke", now.getTime(), startDateMs, 0); + + m = clazz.getMethod("getStartDateSec"); + int startDateSec = (int)m.invoke(o); + assertEquals("reflect getStartDateSec invoke", now.getTime() / 1000, startDateSec, 0); + + m = clazz.getDeclaredMethod("setStartDateMs", Long.class); + m.invoke(o, now.getTime()); + + m = clazz.getMethod("getStartDate"); + startDate = (Date)m.invoke(o); + assertEquals("reflect getStartDate invoke", now, startDate); + + m = clazz.getMethod("getStartDateMs"); + startDateMs = (long)m.invoke(o); + assertEquals("reflect getStartDateMs invoke", now.getTime(), startDateMs, 0); + + m = clazz.getMethod("getStartDateSec"); + startDateSec = (int)m.invoke(o); + assertEquals("reflect getStartDateSec invoke", now.getTime() / 1000, startDateSec, 0); + + m = clazz.getDeclaredMethod("setStartDateSec", Integer.class); + m.invoke(o, (int)(now.getTime() / 1000)); + + now = new Date(now.getTime() / 1000 * 1000); + m = clazz.getMethod("getStartDate"); + startDate = (Date)m.invoke(o); + assertEquals("reflect getStartDate invoke", now, startDate); + + m = clazz.getMethod("getStartDateMs"); + startDateMs = (long)m.invoke(o); + assertEquals("reflect getStartDateMs invoke", now.getTime(), startDateMs, 0); + + m = clazz.getMethod("getStartDateSec"); + startDateSec = (int)m.invoke(o); + assertEquals("reflect getStartDateSec invoke", now.getTime() / 1000, startDateSec, 0); + + } +} From c92ca15e81395d01ece037413d4426d5824b6670 Mon Sep 17 00:00:00 2001 From: Chinmay Kolhatkar Date: Thu, 13 Oct 2016 17:35:16 +0530 Subject: [PATCH 2/2] APEXMALHAR-1818 SQL Support for converting given SQL statement to APEX DAG. Features implemented are: 1. SELECT STATEMENT 2. INSERT STATEMENT 3. INNER JOIN with non-empty equi join condition 4. WHERE clause 5. SCALAR functions implemented in calcite are ready to use 6. Custom scalar functions can be registered. 7. Endpoint can be File OR Kafka OR Streaming Port for both input and output 8. CSV Data Format implemented for both input and output side. 9. Testing on local as well as cluster mode. --- kafka/pom.xml | 7 + .../apex/malhar/kafka/EmbeddedKafka.java | 165 ++++++++ .../lib/transform/TransformOperator.java | 6 +- .../malhar/lib/utils/ClassLoaderUtils.java | 68 ++++ pom.xml | 1 + sql/pom.xml | 213 +++++++++++ .../apex/malhar/sql/SQLExecEnvironment.java | 262 +++++++++++++ .../sql/codegen/BeanClassGenerator.java | 318 ++++++--------- .../sql/codegen/ExpressionCompiler.java | 113 ++++++ .../operators/FilterTransformOperator.java | 71 ++++ .../sql/operators/InnerJoinOperator.java | 54 +++ .../apex/malhar/sql/operators/LineReader.java | 73 ++++ .../malhar/sql/operators/OperatorUtils.java | 79 ++++ .../apex/malhar/sql/planner/ApexRelNode.java | 341 +++++++++++++++++ .../apex/malhar/sql/planner/RelInfo.java | 121 ++++++ .../malhar/sql/planner/RelNodeVisitor.java | 112 ++++++ .../apex/malhar/sql/schema/ApexSQLTable.java | 127 ++++++ .../sql/schema/ApexSQLTableFactory.java | 66 ++++ .../sql/schema/TupleSchemaRegistry.java | 227 +++++++++++ .../malhar/sql/table/CSVMessageFormat.java | 138 +++++++ .../apex/malhar/sql/table/Endpoint.java | 100 +++++ .../apex/malhar/sql/table/FileEndpoint.java | 119 ++++++ .../apex/malhar/sql/table/KafkaEndpoint.java | 136 +++++++ .../apex/malhar/sql/table/MessageFormat.java | 88 +++++ .../apex/malhar/sql/table/StreamEndpoint.java | 147 +++++++ .../apex/malhar/sql/FileEndpointTest.java | 249 ++++++++++++ .../org/apache/apex/malhar/sql/InputPOJO.java | 69 ++++ .../apex/malhar/sql/KafkaEndpointTest.java | 362 ++++++++++++++++++ .../apache/apex/malhar/sql/OutputPOJO.java | 59 +++ .../org/apache/apex/malhar/sql/SerDeTest.java | 223 +++++++++++ .../apex/malhar/sql/StreamEndpointTest.java | 179 +++++++++ .../sql/codegen/BeanClassGeneratorTest.java | 23 +- sql/src/test/resources/input.csv | 6 + sql/src/test/resources/log4j.properties | 50 +++ .../test/resources/model/model_file_csv.json | 27 ++ 35 files changed, 4195 insertions(+), 204 deletions(-) create mode 100644 kafka/src/test/java/org/apache/apex/malhar/kafka/EmbeddedKafka.java create mode 100644 library/src/main/java/org/apache/apex/malhar/lib/utils/ClassLoaderUtils.java create mode 100644 sql/pom.xml create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/SQLExecEnvironment.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/codegen/ExpressionCompiler.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/operators/FilterTransformOperator.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/operators/InnerJoinOperator.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/operators/LineReader.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/operators/OperatorUtils.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/planner/ApexRelNode.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/planner/RelInfo.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/planner/RelNodeVisitor.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/schema/ApexSQLTable.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/schema/ApexSQLTableFactory.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/schema/TupleSchemaRegistry.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/table/CSVMessageFormat.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/table/Endpoint.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/table/FileEndpoint.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/table/KafkaEndpoint.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/table/MessageFormat.java create mode 100644 sql/src/main/java/org/apache/apex/malhar/sql/table/StreamEndpoint.java create mode 100644 sql/src/test/java/org/apache/apex/malhar/sql/FileEndpointTest.java create mode 100644 sql/src/test/java/org/apache/apex/malhar/sql/InputPOJO.java create mode 100644 sql/src/test/java/org/apache/apex/malhar/sql/KafkaEndpointTest.java create mode 100644 sql/src/test/java/org/apache/apex/malhar/sql/OutputPOJO.java create mode 100644 sql/src/test/java/org/apache/apex/malhar/sql/SerDeTest.java create mode 100644 sql/src/test/java/org/apache/apex/malhar/sql/StreamEndpointTest.java create mode 100644 sql/src/test/resources/input.csv create mode 100644 sql/src/test/resources/log4j.properties create mode 100644 sql/src/test/resources/model/model_file_csv.json diff --git a/kafka/pom.xml b/kafka/pom.xml index 5c9aee9c6d..e9683e1154 100755 --- a/kafka/pom.xml +++ b/kafka/pom.xml @@ -242,5 +242,12 @@ ${apex.core.version} jar + + org.apache.kafka + kafka_2.11 + 0.9.0.0 + test + test + diff --git a/kafka/src/test/java/org/apache/apex/malhar/kafka/EmbeddedKafka.java b/kafka/src/test/java/org/apache/apex/malhar/kafka/EmbeddedKafka.java new file mode 100644 index 0000000000..5ddcb18bb4 --- /dev/null +++ b/kafka/src/test/java/org/apache/apex/malhar/kafka/EmbeddedKafka.java @@ -0,0 +1,165 @@ +/** + * 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.apex.malhar.kafka; + +import java.io.File; +import java.io.IOException; +import java.net.ServerSocket; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Properties; + +import org.I0Itec.zkclient.ZkClient; + +import org.apache.commons.io.FileUtils; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; + +import com.google.common.base.Throwables; + +import kafka.admin.AdminUtils; +import kafka.server.KafkaConfig; +import kafka.server.KafkaServer; +import kafka.utils.MockTime; +import kafka.utils.TestUtils; +import kafka.utils.Time; +import kafka.utils.ZKStringSerializer$; +import kafka.utils.ZkUtils; +import kafka.zk.EmbeddedZookeeper; + +public class EmbeddedKafka +{ + private static final String KAFKA_PATH = "/tmp/kafka-test"; + + private ZkClient zkClient; + private ZkUtils zkUtils; + private String BROKERHOST = "127.0.0.1"; + private String BROKERPORT = "9092"; + private EmbeddedZookeeper zkServer; + private KafkaServer kafkaServer; + + public String getBroker() + { + return BROKERHOST + ":" + BROKERPORT; + } + + public void start() throws IOException + { + // Find port + try { + ServerSocket serverSocket = new ServerSocket(0); + BROKERPORT = Integer.toString(serverSocket.getLocalPort()); + serverSocket.close(); + } catch (IOException e) { + throw Throwables.propagate(e); + } + + // Setup Zookeeper + zkServer = new EmbeddedZookeeper(); + String zkConnect = BROKERHOST + ":" + zkServer.port(); + zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer$.MODULE$); + zkUtils = ZkUtils.apply(zkClient, false); + + // Setup brokers + cleanupDir(); + Properties props = new Properties(); + props.setProperty("zookeeper.connect", zkConnect); + props.setProperty("broker.id", "0"); + props.setProperty("log.dirs", KAFKA_PATH); + props.setProperty("listeners", "PLAINTEXT://" + BROKERHOST + ":" + BROKERPORT); + KafkaConfig config = new KafkaConfig(props); + Time mock = new MockTime(); + kafkaServer = TestUtils.createServer(config, mock); + } + + public void stop() throws IOException + { + kafkaServer.shutdown(); + zkClient.close(); + zkServer.shutdown(); + cleanupDir(); + } + + private void cleanupDir() throws IOException + { + FileUtils.deleteDirectory(new File(KAFKA_PATH)); + } + + public void createTopic(String topic) + { + AdminUtils.createTopic(zkUtils, topic, 1, 1, new Properties()); + List servers = new ArrayList(); + servers.add(kafkaServer); + TestUtils.waitUntilMetadataIsPropagated(scala.collection.JavaConversions.asScalaBuffer(servers), topic, 0, 30000); + } + + public void publish(String topic, List messages) + { + Properties producerProps = new Properties(); + producerProps.setProperty("bootstrap.servers", BROKERHOST + ":" + BROKERPORT); + producerProps.setProperty("key.serializer","org.apache.kafka.common.serialization.IntegerSerializer"); + producerProps.setProperty("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer"); + + try (KafkaProducer producer = new KafkaProducer<>(producerProps)) { + for (String message : messages) { + ProducerRecord data = new ProducerRecord<>(topic, message.getBytes(StandardCharsets.UTF_8)); + producer.send(data); + } + } + + List servers = new ArrayList(); + servers.add(kafkaServer); + TestUtils.waitUntilMetadataIsPropagated(scala.collection.JavaConversions.asScalaBuffer(servers), topic, 0, 30000); + } + + public List consume(String topic, int timeout) + { + return consume(topic, timeout, true); + } + + public List consume(String topic, int timeout, boolean earliest) + { + Properties consumerProps = new Properties(); + consumerProps.setProperty("bootstrap.servers", BROKERHOST + ":" + BROKERPORT); + consumerProps.setProperty("group.id", "group0"); + consumerProps.setProperty("client.id", "consumer0"); + consumerProps.setProperty("key.deserializer","org.apache.kafka.common.serialization.IntegerDeserializer"); + consumerProps.setProperty("value.deserializer", "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + consumerProps.put("auto.offset.reset", earliest ? "earliest" : "latest"); // to make sure the consumer starts from the beginning of the topic + KafkaConsumer consumer = new KafkaConsumer<>(consumerProps); + consumer.subscribe(Arrays.asList(topic)); + + List messages = new ArrayList<>(); + + ConsumerRecords records = consumer.poll(timeout); + for (ConsumerRecord record : records) { + messages.add(new String(record.value())); + } + + consumer.close(); + + return messages; + } + +} diff --git a/library/src/main/java/com/datatorrent/lib/transform/TransformOperator.java b/library/src/main/java/com/datatorrent/lib/transform/TransformOperator.java index 483748e613..6ed88192c8 100644 --- a/library/src/main/java/com/datatorrent/lib/transform/TransformOperator.java +++ b/library/src/main/java/com/datatorrent/lib/transform/TransformOperator.java @@ -56,12 +56,12 @@ public class TransformOperator extends BaseOperator implements Operator.Activati { @NotNull private Map expressionMap = new HashMap<>(); - private List expressionFunctions = new LinkedList<>(); + protected List expressionFunctions = new LinkedList<>(); private boolean copyMatchingFields = true; private transient Map transformationMap = new HashMap<>(); - private Class inputClass; - private Class outputClass; + protected Class inputClass; + protected Class outputClass; public TransformOperator() { diff --git a/library/src/main/java/org/apache/apex/malhar/lib/utils/ClassLoaderUtils.java b/library/src/main/java/org/apache/apex/malhar/lib/utils/ClassLoaderUtils.java new file mode 100644 index 0000000000..855c21f2f8 --- /dev/null +++ b/library/src/main/java/org/apache/apex/malhar/lib/utils/ClassLoaderUtils.java @@ -0,0 +1,68 @@ +/** + * 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.apex.malhar.lib.utils; + +import java.io.IOException; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FSDataInputStream; + +@InterfaceStability.Evolving +public class ClassLoaderUtils +{ + /** + * Given the class name it reads and loads the class from the input stream. + * + * @param fqcn fully qualified class name. + * @param inputStream stream from which class is read. + * @return loaded class + * @throws IOException + */ + public static Class readBeanClass(String fqcn, FSDataInputStream inputStream) throws IOException + { + byte[] bytes = IOUtils.toByteArray(inputStream); + inputStream.close(); + return new ByteArrayClassLoader().defineClass(fqcn, bytes); + } + + /** + * Given the class name it reads and loads the class from given byte array. + * + * @param fqcn fully qualified class name. + * @param inputClass byte[] from which class is read. + * @return loaded class + * @throws IOException + */ + public static Class readBeanClass(String fqcn, byte[] inputClass) throws IOException + { + return new ByteArrayClassLoader().defineClass(fqcn, inputClass); + } + + /** + * Byte Array class loader for loading class from byte[] + */ + public static class ByteArrayClassLoader extends ClassLoader + { + Class defineClass(String name, byte[] ba) + { + return defineClass(name, ba, 0, ba.length); + } + } +} diff --git a/pom.xml b/pom.xml index 7a392d0043..da7c38bf2d 100644 --- a/pom.xml +++ b/pom.xml @@ -204,6 +204,7 @@ benchmark apps samples + sql diff --git a/sql/pom.xml b/sql/pom.xml new file mode 100644 index 0000000000..24ef44ca79 --- /dev/null +++ b/sql/pom.xml @@ -0,0 +1,213 @@ + + + 4.0.0 + + + org.apache.apex + malhar + 3.6.0-SNAPSHOT + + + malhar-sql + Apache Apex Malhar SQL Support + jar + + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + test-jar + + package + + + + + + + + + org.apache.apex + apex-common + ${apex.core.version} + provided + + + org.apache.apex + malhar-library + ${project.parent.version} + + + * + * + + + + + org.apache.apex + malhar-contrib + ${project.parent.version} + + + * + * + + + + + org.apache.apex + malhar-kafka + ${project.parent.version} + + + * + * + + + + + org.apache.calcite + calcite-core + 1.7.0 + + + org.apache.calcite.avatica + avatica-metrics + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + com.google.protobuf + protobuf-java + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + commons-dbcp + commons-dbcp + + + org.pentaho + pentaho-aggdesigner-algorithm + + + + + + + + org.apache.xbean + xbean-asm5-shaded + 4.3 + provided + + + + + net.sf.supercsv + super-csv + 2.4.0 + + + com.github.fge + json-schema-validator + 2.0.1 + + + + + org.apache.kafka + kafka_2.11 + 0.9.0.0 + true + + + org.slf4j + slf4j-log4j12 + + + org.slf4j + slf4j-simple + + + log4j + log4j + + + org.apache.zookeeper + zookeeper + + + + + org.apache.kafka + kafka-clients + 0.9.0.0 + + + + + org.apache.apex + malhar-kafka + ${project.parent.version} + test-jar + test + + + * + * + + + + + org.apache.kafka + kafka_2.11 + 0.9.0.0 + test + test + + + + diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/SQLExecEnvironment.java b/sql/src/main/java/org/apache/apex/malhar/sql/SQLExecEnvironment.java new file mode 100644 index 0000000000..de146f65f1 --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/SQLExecEnvironment.java @@ -0,0 +1,262 @@ +/** + * 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.apex.malhar.sql; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Properties; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.sql.planner.RelNodeVisitor; +import org.apache.apex.malhar.sql.schema.ApexSQLTable; +import org.apache.apex.malhar.sql.table.Endpoint; +import org.apache.calcite.config.Lex; +import org.apache.calcite.jdbc.CalciteConnection; +import org.apache.calcite.jdbc.CalciteSchema; +import org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.prepare.CalciteCatalogReader; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.calcite.schema.Function; +import org.apache.calcite.schema.ScalarFunction; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.schema.Table; +import org.apache.calcite.schema.impl.ScalarFunctionImpl; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperatorTable; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.parser.SqlParser; +import org.apache.calcite.sql.util.ChainedSqlOperatorTable; +import org.apache.calcite.tools.FrameworkConfig; +import org.apache.calcite.tools.Frameworks; +import org.apache.calcite.tools.Planner; +import org.apache.calcite.util.Util; +import org.apache.hadoop.classification.InterfaceStability; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; + +import com.datatorrent.api.DAG; + +/** + * SQL Execution Environment for SQL integration API of Apex. + * This exposes calcite functionality in simple way. Most of the APIs are with builder pattern which makes it + * easier to construct a DAG using this object. + * + * Eg. + *
+ * SQLExecEnvironment.getEnvironment(dag)
+ *                   .registerTable("TABLE1", Object of type {@link Endpoint})
+ *                   .registerTable("TABLE2", Object of type {@link Endpoint})
+ *                   .executeSQL("INSERT INTO TABLE2 SELECT STREAM * FROM TABLE1);
+ * 
+ * + * Above code will evaluate SQL statement and convert the resultant Relational Algebra to a sub-DAG. + */ +@InterfaceStability.Evolving +public class SQLExecEnvironment +{ + private static final Logger logger = LoggerFactory.getLogger(SQLExecEnvironment.class); + + private final JavaTypeFactoryImpl typeFactory; + private SchemaPlus schema = Frameworks.createRootSchema(true); + + /** + * Construct SQL Execution Environment which works on given DAG objec. + */ + private SQLExecEnvironment() + { + this.typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT); + } + + /** + * Given SQLExec{@link SQLExecEnvironment} object for given {@link DAG}. + * + * @return Returns {@link SQLExecEnvironment} object + */ + public static SQLExecEnvironment getEnvironment() + { + return new SQLExecEnvironment(); + } + + /** + * Use given model file to initialize {@link SQLExecEnvironment}. + * The model file contains definitions of endpoints and data formats. + * Example of file format is like following: + *
+   *   {
+   *     version: '1.0',
+   *     defaultSchema: 'APEX',
+   *     schemas: [{
+   *       name: 'APEX',
+   *       tables: [
+   *         {
+   *            name: 'ORDERS',
+   *            type: 'custom',
+   *            factory: 'org.apache.apex.malhar.sql.schema.ApexSQLTableFactory',
+   *            stream: {
+   *              stream: true
+   *            },
+   *            operand: {
+   *              endpoint: 'file',
+   *              messageFormat: 'csv',
+   *              endpointOperands: {
+   *                directory: '/tmp/input'
+   *              },
+   *              messageFormatOperands: {
+   *                schema: '{"separator":",","quoteChar":"\\"","fields":[{"name":"RowTime","type":"Date","constraints":{"format":"dd/MM/yyyy hh:mm:ss"}},{"name":"id","type":"Integer"},{"name":"Product","type":"String"},{"name":"units","type":"Integer"}]}'
+   *            }
+   *            }
+   *         }
+   *       ]
+   *     }]
+   *   }
+   * 
+ * + * @param model String content of model file. + * @return Returns this {@link SQLExecEnvironment} + */ + public SQLExecEnvironment withModel(String model) + { + if (model == null) { + return this; + } + + Properties p = new Properties(); + p.put("model", "inline:" + model); + try (Connection connection = DriverManager.getConnection("jdbc:calcite:", p)) { + CalciteConnection conn = connection.unwrap(CalciteConnection.class); + this.schema = conn.getRootSchema().getSubSchema(connection.getSchema()); + } catch (SQLException e) { + throw new RuntimeException(e); + } + + return this; + } + + /** + * Register a table using {@link Endpoint} with this {@link SQLExecEnvironment} + * + * @param name Name of the table that needs to be present with SQL Statement. + * @param endpoint Object of type {@link Endpoint} + * + * @return Returns this {@link SQLExecEnvironment} + */ + public SQLExecEnvironment registerTable(String name, Endpoint endpoint) + { + Preconditions.checkNotNull(name, "Table name cannot be null"); + registerTable(name, new ApexSQLTable(schema, name, endpoint)); + return this; + } + + /** + * Register a table using {@link Table} with this {@link SQLExecEnvironment} + * + * @param name Name of the table that needs to be present with SQL statement + * @param table Object of type {@link Table} + * + * @return Returns this {@link SQLExecEnvironment} + */ + public SQLExecEnvironment registerTable(String name, Table table) + { + Preconditions.checkNotNull(name, "Table name cannot be null"); + schema.add(name, table); + return this; + } + + /** + * Register custom function with this {@link SQLExecEnvironment} + * + * @param name Name of the scalar SQL function that needs made available to SQL Statement + * @param fn Object of type {@link Function} + * + * @return Returns this {@link SQLExecEnvironment} + */ + public SQLExecEnvironment registerFunction(String name, Function fn) + { + Preconditions.checkNotNull(name, "Function name cannot be null"); + schema.add(name, fn); + return this; + } + + /** + * Register custom function from given static method with this {@link SQLExecEnvironment} + * + * @param name Name of the scalar SQL function that needs make available to SQL Statement + * @param clazz {@link Class} which contains given static method + * @param methodName Name of the method from given clazz + * + * @return Return this {@link SQLExecEnvironment} + */ + public SQLExecEnvironment registerFunction(String name, Class clazz, String methodName) + { + Preconditions.checkNotNull(name, "Function name cannot be null"); + ScalarFunction scalarFunction = ScalarFunctionImpl.create(clazz, methodName); + return registerFunction(name, scalarFunction); + } + + /** + * This is the main method takes SQL statement as input and contructs a DAG using contructs registered with this + * {@link SQLExecEnvironment}. + * + * @param sql SQL statement that should be converted to a DAG. + */ + public void executeSQL(DAG dag, String sql) + { + FrameworkConfig config = buildFrameWorkConfig(); + Planner planner = Frameworks.getPlanner(config); + try { + logger.info("Parsing SQL statement: {}", sql); + SqlNode parsedTree = planner.parse(sql); + SqlNode validatedTree = planner.validate(parsedTree); + RelNode relationalTree = planner.rel(validatedTree).rel; + logger.info("RelNode relationalTree generate from SQL statement is:\n {}", + Util.toLinux(RelOptUtil.toString(relationalTree))); + RelNodeVisitor visitor = new RelNodeVisitor(dag, typeFactory); + visitor.traverse(relationalTree); + } catch (Exception e) { + throw Throwables.propagate(e); + } finally { + planner.close(); + } + } + + /** + * Method method build a calcite framework configuration for calcite to parse SQL and generate relational tree + * out of it. + * @return FrameworkConfig + */ + private FrameworkConfig buildFrameWorkConfig() + { + List sqlOperatorTables = new ArrayList<>(); + sqlOperatorTables.add(SqlStdOperatorTable.instance()); + sqlOperatorTables + .add(new CalciteCatalogReader(CalciteSchema.from(schema), false, Collections.emptyList(), typeFactory)); + return Frameworks.newConfigBuilder().defaultSchema(schema) + .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()) + .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables)).build(); + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/codegen/BeanClassGenerator.java b/sql/src/main/java/org/apache/apex/malhar/sql/codegen/BeanClassGenerator.java index ffeafb509d..3e384a205c 100644 --- a/sql/src/main/java/org/apache/apex/malhar/sql/codegen/BeanClassGenerator.java +++ b/sql/src/main/java/org/apache/apex/malhar/sql/codegen/BeanClassGenerator.java @@ -25,9 +25,7 @@ import org.codehaus.jettison.json.JSONException; import org.apache.apex.malhar.sql.schema.TupleSchemaRegistry; -import org.apache.commons.io.IOUtils; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.xbean.asm5.ClassWriter; import org.apache.xbean.asm5.Opcodes; @@ -55,16 +53,26 @@ public class BeanClassGenerator { public static final ImmutableMap PRIMITIVE_TYPES; + public static final char typeIdentifierBoolean = 'Z'; + public static final char typeIdentifierChar = 'C'; + public static final char typeIdentifierByte = 'B'; + public static final char typeIdentifierShort = 'S'; + public static final char typeIdentifierInt = 'I'; + public static final char typeIdentifierFloat = 'F'; + public static final char typeIdentifierLong = 'J'; + public static final char typeIdentifierDouble = 'D'; + + static { Map types = Maps.newHashMap(); - types.put("boolean", 'Z'); - types.put("char", 'C'); - types.put("byte", 'B'); - types.put("short", 'S'); - types.put("int", 'I'); - types.put("float", 'F'); - types.put("long", 'J'); - types.put("double", 'D'); + types.put("boolean", typeIdentifierBoolean); + types.put("char", typeIdentifierChar); + types.put("byte", typeIdentifierByte); + types.put("short", typeIdentifierShort); + types.put("int", typeIdentifierInt); + types.put("float", typeIdentifierFloat); + types.put("long", typeIdentifierLong); + types.put("double", typeIdentifierDouble); PRIMITIVE_TYPES = ImmutableMap.copyOf(types); } @@ -100,7 +108,7 @@ public static byte[] createAndWriteBeanClass(String fqcn, List", "()V", null, null); @@ -180,6 +170,95 @@ private static void addDefaultConstructor(ClassNode classNode) classNode.methods.add(constructorNode); } + /** + * Add private field to the class + * @param classNode ClassNode which needs to be populated with private field. + * @param fieldName Name of the field + * @param fieldJavaType Java ASM type of the field + */ + @SuppressWarnings("unchecked") + private static void addPrivateField(ClassNode classNode, String fieldName, String fieldJavaType) + { + FieldNode fieldNode = new FieldNode(Opcodes.ACC_PRIVATE, fieldName, fieldJavaType, null, null); + classNode.fields.add(fieldNode); + } + + /** + * Add public getter method for given field + * @param classNode ClassNode which needs to be populated with public getter. + * @param fieldName Name of the field for which public getter needs to be added. + * @param fieldNameForMethods Suffix of the getter method, Prefix "is" or "get" is added by this method. + * @param fieldJavaType Java ASM type of the field + */ + @SuppressWarnings("unchecked") + private static void addGetter(ClassNode classNode, String fieldName, String fieldNameForMethods, String fieldJavaType) + { + String getterSignature = "()" + fieldJavaType; + MethodNode getterNode = new MethodNode(Opcodes.ACC_PUBLIC, + (fieldJavaType.equals(typeIdentifierBoolean) ? "is" : "get") + fieldNameForMethods, + getterSignature, null, null); + getterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + getterNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); + + int returnOpCode; + if (fieldJavaType.equals(Character.toString(typeIdentifierBoolean)) || + fieldJavaType.equals(Character.toString(typeIdentifierByte)) || + fieldJavaType.equals(Character.toString(typeIdentifierChar)) || + fieldJavaType.equals(Character.toString(typeIdentifierShort)) || + fieldJavaType.equals(Character.toString(typeIdentifierInt))) { + returnOpCode = Opcodes.IRETURN; + } else if (fieldJavaType.equals(Character.toString(typeIdentifierLong))) { + returnOpCode = Opcodes.LRETURN; + } else if (fieldJavaType.equals(Character.toString(typeIdentifierFloat))) { + returnOpCode = Opcodes.FRETURN; + } else if (fieldJavaType.equals(Character.toString(typeIdentifierDouble))) { + returnOpCode = Opcodes.DRETURN; + } else { + returnOpCode = Opcodes.ARETURN; + } + getterNode.instructions.add(new InsnNode(returnOpCode)); + + classNode.methods.add(getterNode); + } + + /** + * Add public setter for given field + * @param classNode ClassNode which needs to be populated with public setter + * @param fieldName Name of the field for which setter needs to be added + * @param fieldNameForMethods Suffix for setter method. Prefix "set" is added by this method + * @param fieldJavaType Java ASM type of the field + */ + @SuppressWarnings("unchecked") + private static void addSetter(ClassNode classNode, String fieldName, String fieldNameForMethods, String fieldJavaType) + { + String setterSignature = '(' + fieldJavaType + ')' + 'V'; + MethodNode setterNode = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods, setterSignature, null, + null); + setterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); + + int loadOpCode; + if (fieldJavaType.equals(Character.toString(typeIdentifierBoolean)) || + fieldJavaType.equals(Character.toString(typeIdentifierByte)) || + fieldJavaType.equals(Character.toString(typeIdentifierChar)) || + fieldJavaType.equals(Character.toString(typeIdentifierShort)) || + fieldJavaType.equals(Character.toString(typeIdentifierInt))) { + loadOpCode = Opcodes.ILOAD; + } else if (fieldJavaType.equals(Character.toString(typeIdentifierLong))) { + loadOpCode = Opcodes.LLOAD; + } else if (fieldJavaType.equals(Character.toString(typeIdentifierFloat))) { + loadOpCode = Opcodes.FLOAD; + } else if (fieldJavaType.equals(Character.toString(typeIdentifierDouble))) { + loadOpCode = Opcodes.DLOAD; + } else { + loadOpCode = Opcodes.ALOAD; + } + setterNode.instructions.add(new VarInsnNode(loadOpCode, 1)); + + setterNode.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, fieldJavaType)); + setterNode.instructions.add(new InsnNode(Opcodes.RETURN)); + classNode.methods.add(setterNode); + } + /** * Date field is explicitly handled and provided with 3 variants of types of same data. * 1. java.util.Date format @@ -327,126 +406,6 @@ private static void addDateFields(ClassNode classNode, String fieldName, String classNode.methods.add(setterNodeMs); } - @SuppressWarnings("unchecked") - private static void addIntGetterNSetter(ClassNode classNode, String fieldName, String fieldNameForMethods, - String fieldJavaType, boolean isBoolean) - { - // Create getter - String getterSignature = "()" + fieldJavaType; - MethodNode getterNode = new MethodNode(Opcodes.ACC_PUBLIC, (isBoolean ? "is" : "get") + fieldNameForMethods, - getterSignature, null, null); - getterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); - getterNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); - getterNode.instructions.add(new InsnNode(Opcodes.IRETURN)); - classNode.methods.add(getterNode); - - // Create setter - String setterSignature = '(' + fieldJavaType + ')' + 'V'; - MethodNode setterNode = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods, setterSignature, null, - null); - setterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); - setterNode.instructions.add(new VarInsnNode(Opcodes.ILOAD, 1)); - setterNode.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, fieldJavaType)); - setterNode.instructions.add(new InsnNode(Opcodes.RETURN)); - classNode.methods.add(setterNode); - } - - @SuppressWarnings("unchecked") - private static void addLongGetterNSetter(ClassNode classNode, String fieldName, String fieldNameForMethods, - String fieldJavaType) - { - // Create getter - String getterSignature = "()" + fieldJavaType; - MethodNode getterNode = new MethodNode(Opcodes.ACC_PUBLIC, "get" + fieldNameForMethods, getterSignature, null, - null); - getterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); - getterNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); - getterNode.instructions.add(new InsnNode(Opcodes.LRETURN)); - classNode.methods.add(getterNode); - - // Create setter - String setterSignature = '(' + fieldJavaType + ')' + 'V'; - MethodNode setterNode = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods, setterSignature, null, - null); - setterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); - setterNode.instructions.add(new VarInsnNode(Opcodes.LLOAD, 1)); - setterNode.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, fieldJavaType)); - setterNode.instructions.add(new InsnNode(Opcodes.RETURN)); - classNode.methods.add(setterNode); - } - - @SuppressWarnings("unchecked") - private static void addFloatGetterNSetter(ClassNode classNode, String fieldName, String fieldNameForMethods, - String fieldJavaType) - { - // Create getter - String getterSignature = "()" + fieldJavaType; - MethodNode getterNode = new MethodNode(Opcodes.ACC_PUBLIC, "get" + fieldNameForMethods, getterSignature, null, - null); - getterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); - getterNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); - getterNode.instructions.add(new InsnNode(Opcodes.FRETURN)); - classNode.methods.add(getterNode); - - // Create setter - String setterSignature = '(' + fieldJavaType + ')' + 'V'; - MethodNode setterNode = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods, setterSignature, null, - null); - setterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); - setterNode.instructions.add(new VarInsnNode(Opcodes.FLOAD, 1)); - setterNode.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, fieldJavaType)); - setterNode.instructions.add(new InsnNode(Opcodes.RETURN)); - classNode.methods.add(setterNode); - } - - @SuppressWarnings("unchecked") - private static void addDoubleGetterNSetter(ClassNode classNode, String fieldName, String fieldNameForMethods, - String fieldJavaType) - { - // Create getter - String getterSignature = "()" + fieldJavaType; - MethodNode getterNode = new MethodNode(Opcodes.ACC_PUBLIC, "get" + fieldNameForMethods, getterSignature, null, - null); - getterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); - getterNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); - getterNode.instructions.add(new InsnNode(Opcodes.DRETURN)); - classNode.methods.add(getterNode); - - // Create setter - String setterSignature = '(' + fieldJavaType + ')' + 'V'; - MethodNode setterNode = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods, setterSignature, null, - null); - setterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); - setterNode.instructions.add(new VarInsnNode(Opcodes.DLOAD, 1)); - setterNode.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, fieldJavaType)); - setterNode.instructions.add(new InsnNode(Opcodes.RETURN)); - classNode.methods.add(setterNode); - } - - @SuppressWarnings("unchecked") - private static void addObjectGetterNSetter(ClassNode classNode, String fieldName, String fieldNameForMethods, - String fieldJavaType) - { - // Create getter - String getterSignature = "()" + fieldJavaType; - MethodNode getterNode = new MethodNode(Opcodes.ACC_PUBLIC, "get" + fieldNameForMethods, getterSignature, null, - null); - getterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); - getterNode.instructions.add(new FieldInsnNode(Opcodes.GETFIELD, classNode.name, fieldName, fieldJavaType)); - getterNode.instructions.add(new InsnNode(Opcodes.ARETURN)); - classNode.methods.add(getterNode); - - // Create setter - String setterSignature = '(' + fieldJavaType + ')' + 'V'; - MethodNode setterNode = new MethodNode(Opcodes.ACC_PUBLIC, "set" + fieldNameForMethods, setterSignature, null, - null); - setterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 0)); - setterNode.instructions.add(new VarInsnNode(Opcodes.ALOAD, 1)); - setterNode.instructions.add(new FieldInsnNode(Opcodes.PUTFIELD, classNode.name, fieldName, fieldJavaType)); - setterNode.instructions.add(new InsnNode(Opcodes.RETURN)); - classNode.methods.add(setterNode); - } - /** * Adds a toString method to underlying class. Uses StringBuilder to generate the final string. * @@ -488,7 +447,8 @@ private static void addToStringMethod(ClassNode classNode, List readBeanClass(String fqcn, FSDataInputStream inputStream) throws IOException - { - byte[] bytes = IOUtils.toByteArray(inputStream); - inputStream.close(); - return new ByteArrayClassLoader().defineClass(fqcn, bytes); - } - - /** - * Given the class name it reads and loads the class from given byte array. - * - * @param fqcn fully qualified class name. - * @param inputClass byte[] from which class is read. - * @return loaded class - * @throws IOException - */ - public static Class readBeanClass(String fqcn, byte[] inputClass) throws IOException - { - return new ByteArrayClassLoader().defineClass(fqcn, inputClass); - } - - private static class ByteArrayClassLoader extends ClassLoader - { - Class defineClass(String name, byte[] ba) - { - return defineClass(name, ba, 0, ba.length); - } - } } diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/codegen/ExpressionCompiler.java b/sql/src/main/java/org/apache/apex/malhar/sql/codegen/ExpressionCompiler.java new file mode 100644 index 0000000000..e12ff53a94 --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/codegen/ExpressionCompiler.java @@ -0,0 +1,113 @@ +/** + * 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.apex.malhar.sql.codegen; + +import java.util.List; +import java.util.regex.Matcher; + +import org.apache.apex.malhar.sql.operators.OperatorUtils; +import org.apache.calcite.adapter.enumerable.JavaRowFormat; +import org.apache.calcite.adapter.enumerable.PhysType; +import org.apache.calcite.adapter.enumerable.PhysTypeImpl; +import org.apache.calcite.adapter.enumerable.RexToLixTranslator; +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.linq4j.function.Function1; +import org.apache.calcite.linq4j.tree.BlockBuilder; +import org.apache.calcite.linq4j.tree.BlockStatement; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexProgram; +import org.apache.calcite.rex.RexProgramBuilder; +import org.apache.calcite.util.Pair; +import org.apache.hadoop.classification.InterfaceStability; + +import com.google.common.collect.ImmutableList; + +/** + * Converts calcite expression of type {@link RexNode} to quasi-Java expression which can be used + * with {@link com.datatorrent.lib.util.PojoUtils} + */ +@InterfaceStability.Evolving +public class ExpressionCompiler +{ + private final RexBuilder rexBuilder; + + public ExpressionCompiler(RexBuilder rexBuilder) + { + this.rexBuilder = rexBuilder; + } + + /** + * Create quasi-Java expression from given {@link RexNode} + * + * @param node Expression in the form of {@link RexNode} + * @param inputRowType Input Data type to expression in the form of {@link RelDataType} + * @param outputRowType Output data type of expression in the form of {@link RelDataType} + * + * @return Returns quasi-Java expression + */ + public String getExpression(RexNode node, RelDataType inputRowType, RelDataType outputRowType) + { + final RexProgramBuilder programBuilder = new RexProgramBuilder(inputRowType, rexBuilder); + programBuilder.addProject(node, null); + final RexProgram program = programBuilder.getProgram(); + + final BlockBuilder builder = new BlockBuilder(); + final JavaTypeFactory javaTypeFactory = (JavaTypeFactory)rexBuilder.getTypeFactory(); + + final RexToLixTranslator.InputGetter inputGetter = new RexToLixTranslator.InputGetterImpl(ImmutableList + .of(Pair.of(Expressions.variable(Object[].class, "inputValues"), + PhysTypeImpl.of(javaTypeFactory, inputRowType, JavaRowFormat.ARRAY, false)))); + final Function1 correlates = + new Function1() + { + public RexToLixTranslator.InputGetter apply(String a0) + { + throw new UnsupportedOperationException(); + } + }; + + final List list = RexToLixTranslator.translateProjects(program, javaTypeFactory, builder, + PhysTypeImpl.of(javaTypeFactory, outputRowType, JavaRowFormat.ARRAY, false), null, inputGetter, correlates); + + for (int i = 0; i < list.size(); i++) { + Statement statement = Expressions.statement(list.get(i)); + builder.add(statement); + } + + return finalizeExpression(builder.toBlock(), inputRowType); + } + + private String finalizeExpression(BlockStatement blockStatement, RelDataType inputRowType) + { + String s = Expressions.toString(blockStatement.statements.get(0)); + int idx = 0; + for (RelDataTypeField field : inputRowType.getFieldList()) { + String fieldName = OperatorUtils.getFieldName(field); + s = s.replaceAll(String.format("inputValues\\[%d\\]", idx++), "\\{\\$." + Matcher.quoteReplacement(fieldName) + "\\}"); + } + + return s.substring(0, s.length() - 2); + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/operators/FilterTransformOperator.java b/sql/src/main/java/org/apache/apex/malhar/sql/operators/FilterTransformOperator.java new file mode 100644 index 0000000000..192c58b7dc --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/operators/FilterTransformOperator.java @@ -0,0 +1,71 @@ +/** + * 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.apex.malhar.sql.operators; + +import org.apache.hadoop.classification.InterfaceStability; + +import com.datatorrent.api.Context; +import com.datatorrent.lib.expression.Expression; +import com.datatorrent.lib.transform.TransformOperator; +import com.datatorrent.lib.util.PojoUtils; + +/** + * This is an extension of {@link TransformOperator} which also takes care of filtering tuples. + */ +@InterfaceStability.Evolving +public class FilterTransformOperator extends TransformOperator +{ + private String condition; + private transient Expression conditionExpression; + + @Override + public void activate(Context context) + { + super.activate(context); + if (condition != null) { + conditionExpression = PojoUtils.createExpression(inputClass, condition, Boolean.class, + expressionFunctions.toArray(new String[expressionFunctions.size()])); + } + } + + @Override + protected void processTuple(Object in) + { + if ((conditionExpression != null) && (conditionExpression.execute(in) == Boolean.FALSE)) { + return; + } + + super.processTuple(in); + } + + public String getCondition() + { + return condition; + } + + /** + * Set quasi-Java expression which acts as filtering logic for given tuple + * + * @param condition Expression which can be evaluated for filtering + */ + public void setCondition(String condition) + { + this.condition = condition; + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/operators/InnerJoinOperator.java b/sql/src/main/java/org/apache/apex/malhar/sql/operators/InnerJoinOperator.java new file mode 100644 index 0000000000..0b21898b38 --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/operators/InnerJoinOperator.java @@ -0,0 +1,54 @@ +/** + * 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.apex.malhar.sql.operators; + +import org.apache.apex.malhar.lib.join.POJOInnerJoinOperator; +import org.apache.hadoop.classification.InterfaceStability; + +import com.datatorrent.api.Context; + +/** + * This is an extension of {@link POJOInnerJoinOperator} operator which works over a global scope and + * does not have time bound expiry of join tuples. + */ +@InterfaceStability.Evolving +public class InnerJoinOperator extends POJOInnerJoinOperator +{ + private long time = System.currentTimeMillis(); + + @Override + public void setup(Context.OperatorContext context) + { + this.setExpiryTime(1L); + // Number of buckets is set to 47000 because this is rounded number closer to sqrt of MAXINT. This guarantees + // even distribution of keys across buckets. + this.setNoOfBuckets(47000); + this.setTimeFieldsStr(""); + super.setup(context); + } + + @Override + public long extractTime(Object tuple, boolean isStream1Data) + { + /** + * Return extract time which is always more than time when the operator is started. + */ + return time + 3600000L; + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/operators/LineReader.java b/sql/src/main/java/org/apache/apex/malhar/sql/operators/LineReader.java new file mode 100644 index 0000000000..b0225f0fc8 --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/operators/LineReader.java @@ -0,0 +1,73 @@ +/** + * 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.apex.malhar.sql.operators; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; + +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; + +import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.lib.io.fs.AbstractFileInputOperator; + +/** + * This operator reads data from given file/folder in line by line fashion. + */ +@InterfaceStability.Evolving +public class LineReader extends AbstractFileInputOperator +{ + public final transient DefaultOutputPort output = new DefaultOutputPort<>(); + + protected transient BufferedReader br; + + @Override + protected InputStream openFile(Path path) throws IOException + { + InputStream is = super.openFile(path); + br = new BufferedReader(new InputStreamReader(is)); + return is; + } + + @Override + protected void closeFile(InputStream is) throws IOException + { + super.closeFile(is); + br.close(); + br = null; + } + + @Override + protected byte[] readEntity() throws IOException + { + String s = br.readLine(); + if (s != null) { + return s.getBytes(); + } + return null; + } + + @Override + protected void emit(byte[] tuple) + { + output.emit(tuple); + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/operators/OperatorUtils.java b/sql/src/main/java/org/apache/apex/malhar/sql/operators/OperatorUtils.java new file mode 100644 index 0000000000..e5a19280fe --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/operators/OperatorUtils.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.apex.malhar.sql.operators; + +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.hadoop.classification.InterfaceStability; + +@InterfaceStability.Evolving +public class OperatorUtils +{ + private static int opCount = 1; + private static int streamCount = 1; + + /** + * This method generates unique name for the operator. + * + * @param operatorType Base name of the operator. + * @return Returns unique name of the operator. + */ + public static String getUniqueOperatorName(String operatorType) + { + return operatorType + "_" + opCount++; + } + + /** + * This method generates unique name for the stream using input and output. + * + * @param output Name of the output end of the stream + * @param input Name of the input end of the stream + * @return Returns unique name of the stream. + */ + public static String getUniqueStreamName(String output, String input) + { + return output + "_" + input + "_" + streamCount++; + } + + /** + * This method gives field name for POJO class for given {@link RelDataTypeField} object. + * + * @param field field object that needs to be converted to POJO class field name + * @return Return field name from POJO class + */ + public static String getFieldName(RelDataTypeField field) + { + SqlTypeName sqlTypeName = field.getType().getSqlTypeName(); + String name = getValidFieldName(field); + + name = (sqlTypeName == SqlTypeName.TIMESTAMP) ? + (name + "Ms") : + ((sqlTypeName == SqlTypeName.DATE) ? (name + "Sec") : name); + + return name; + } + + public static String getValidFieldName(RelDataTypeField field) + { + String name = field.getName().replaceAll("\\W", ""); + name = (name.length() == 0) ? "f0" : name; + return name; + } + +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/planner/ApexRelNode.java b/sql/src/main/java/org/apache/apex/malhar/sql/planner/ApexRelNode.java new file mode 100644 index 0000000000..7a0d339da7 --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/planner/ApexRelNode.java @@ -0,0 +1,341 @@ +/** + * 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.apex.malhar.sql.planner; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.apex.malhar.sql.codegen.ExpressionCompiler; +import org.apache.apex.malhar.sql.operators.FilterTransformOperator; +import org.apache.apex.malhar.sql.operators.InnerJoinOperator; +import org.apache.apex.malhar.sql.operators.OperatorUtils; +import org.apache.apex.malhar.sql.schema.ApexSQLTable; +import org.apache.apex.malhar.sql.schema.TupleSchemaRegistry; +import org.apache.apex.malhar.sql.table.Endpoint; + +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.TableModify; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.logical.LogicalFilter; +import org.apache.calcite.rel.logical.LogicalJoin; +import org.apache.calcite.rel.logical.LogicalProject; +import org.apache.calcite.rel.logical.LogicalTableModify; +import org.apache.calcite.rel.logical.LogicalTableScan; +import org.apache.calcite.rel.stream.Delta; +import org.apache.calcite.rel.stream.LogicalDelta; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.util.Pair; +import org.apache.hadoop.classification.InterfaceStability; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; + +import com.datatorrent.api.Context; +import com.datatorrent.api.DAG; +import com.datatorrent.api.Operator; +import com.datatorrent.lib.io.ConsoleOutputOperator; + +/** + * This class defines how to populate DAG of Apex for the relational nodes of SQL Calcite + */ +@InterfaceStability.Evolving +public abstract class ApexRelNode +{ + public static Map relNodeMapping = ImmutableMap.builder() + .put(LogicalDelta.class, new ApexDeltaRel()) + .put(LogicalTableScan.class, new ApexTableScanRel()) + .put(LogicalTableModify.class, new ApexTableModifyRel()) + .put(LogicalProject.class, new ApexProjectRel()) + .put(LogicalFilter.class, new ApexFilterRel()) + .put(LogicalJoin.class, new ApexJoinRel()) + .build(); + + public abstract RelInfo visit(RelContext context, RelNode node, List inputStreams); + + public static class RelContext + { + public DAG dag; + public JavaTypeFactory typeFactory; + public TupleSchemaRegistry schemaRegistry; + + public RelContext(DAG dag, JavaTypeFactory typeFactory, TupleSchemaRegistry registry) + { + this.dag = dag; + this.typeFactory = typeFactory; + this.schemaRegistry = registry; + } + } + + /** + * This is visitor for {@link Delta} to emit the data to {@link ConsoleOutputOperator}. + */ + private static class ApexDeltaRel extends ApexRelNode + { + @Override + public RelInfo visit(RelContext context, RelNode node, List inputStreams) + { + Delta delta = (Delta)node; + + ConsoleOutputOperator console = context.dag + .addOperator(OperatorUtils.getUniqueOperatorName(delta.getRelTypeName()), ConsoleOutputOperator.class); + console.setStringFormat("Delta Record: %s"); + + return new RelInfo("Delta", Lists.newArrayList(console.input), console, null, + delta.getRowType()); + } + } + + /** + * This is visitor for {@link TableScan} for adding operators to DAG. + */ + private static class ApexTableScanRel extends ApexRelNode + { + @Override + public RelInfo visit(RelContext context, RelNode node, List inputStreams) + { + TableScan scan = (TableScan)node; + ApexSQLTable table = scan.getTable().unwrap(ApexSQLTable.class); + Endpoint endpoint = table.getEndpoint(); + return endpoint.populateInputDAG(context.dag, context.typeFactory); + } + } + + /** + * This is visitor for {@link TableModify} for adding operators to DAG. + */ + private static class ApexTableModifyRel extends ApexRelNode + { + @Override + public RelInfo visit(RelContext context, RelNode node, List inputStreams) + { + /** + * Only INSERT is allowed as it representation destination for DAG processing. Other types like UPDATE, DELETE, + * MERGE does not represent the same. + */ + + TableModify modify = (TableModify)node; + Preconditions.checkArgument(modify.isInsert(), "Only INSERT allowed for table modify"); + + ApexSQLTable table = modify.getTable().unwrap(ApexSQLTable.class); + + Endpoint endpoint = table.getEndpoint(); + return endpoint.populateOutputDAG(context.dag, context.typeFactory); + } + } + + /** + * This is visitor for {@link Project} for adding operators to DAG. + */ + private static class ApexProjectRel extends ApexRelNode + { + @Override + public RelInfo visit(RelContext context, RelNode node, List inputStreams) + { + Project project = (Project)node; + if (inputStreams.size() == 0 || inputStreams.size() > 1) { + throw new UnsupportedOperationException("Project is a SingleRel"); + } + + FilterTransformOperator operator = context.dag + .addOperator(OperatorUtils.getUniqueOperatorName(project.getRelTypeName()), FilterTransformOperator.class); + Map expMap = new HashMap<>(); + ExpressionCompiler compiler = new ExpressionCompiler(new RexBuilder(project.getCluster().getTypeFactory())); + + for (Pair pair : Pair.zip(project.getRowType().getFieldList(), + project.getProjects())) { + String fieldName = OperatorUtils.getFieldName(pair.left); + String expression = compiler.getExpression(pair.right, project.getInput().getRowType(), project.getRowType()); + expMap.put(fieldName, expression); + } + operator.setExpressionMap(expMap); + + return new RelInfo("Project", Lists.newArrayList(operator.input), operator, operator.output, + project.getRowType()); + } + } + + /** + * This is visitor for {@link Filter} for adding operators to DAG. + */ + private static class ApexFilterRel extends ApexRelNode + { + @Override + public RelInfo visit(RelContext context, RelNode node, List inputStreams) + { + Filter filter = (Filter)node; + if (inputStreams.size() == 0 || inputStreams.size() > 1) { + throw new UnsupportedOperationException("Filter is a SingleRel"); + } + + FilterTransformOperator operator = context.dag + .addOperator(OperatorUtils.getUniqueOperatorName(filter.getRelTypeName()), FilterTransformOperator.class); + ExpressionCompiler compiler = new ExpressionCompiler(new RexBuilder(filter.getCluster().getTypeFactory())); + String expression = compiler.getExpression(filter.getCondition(), filter.getInput().getRowType(), + filter.getRowType()); + + Map expMap = new HashMap<>(); + for (Pair pair : Pair.zip(filter.getInput().getRowType().getFieldList(), + filter.getRowType().getFieldList())) { + String leftName = OperatorUtils.getFieldName(pair.left); + String rightName = OperatorUtils.getFieldName(pair.right); + expMap.put(leftName, rightName); + } + operator.setExpressionMap(expMap); + operator.setCondition(expression); + + return new RelInfo("Filter", Lists.newArrayList(operator.input), operator, operator.output, + filter.getRowType()); + } + } + + /** + * This is visitor for {@link Join} for adding operators to DAG. + */ + private static class ApexJoinRel extends ApexRelNode + { + + @Override + public RelInfo visit(RelContext context, RelNode node, List inputStreams) + { + Join join = (Join)node; + if (inputStreams.size() != 2) { + throw new UnsupportedOperationException("Join is a BiRel"); + } + + if ((join.getJoinType() == JoinRelType.FULL) || (join.getJoinType() == JoinRelType.LEFT) || + (join.getJoinType() == JoinRelType.RIGHT)) { + throw new UnsupportedOperationException("Outer joins are not supported"); + } + + final List leftKeys = new ArrayList<>(); + final List rightKeys = new ArrayList<>(); + + RexNode remaining = + RelOptUtil.splitJoinCondition(join.getLeft(), join.getRight(), join.getCondition(), leftKeys, rightKeys); + + if (leftKeys.size() != rightKeys.size()) { + throw new RuntimeException("Unexpected condition reached. Left and right condition count should be same"); + } + + if (leftKeys.size() == 0) { + throw new UnsupportedOperationException("Theta joins are not supported."); + } + + RelInfo relInfo = addInnerJoinOperator(join, leftKeys, rightKeys, context); + + if (!remaining.isAlwaysTrue()) { + relInfo = addJoinFilter(join, remaining, relInfo, context); + } + + return relInfo; + } + + private RelInfo addJoinFilter(Join join, RexNode remaining, RelInfo relInfo, RelContext context) + { + FilterTransformOperator operator = context.dag + .addOperator(OperatorUtils.getUniqueOperatorName(join.getRelTypeName() + "_Filter"), + FilterTransformOperator.class); + ExpressionCompiler compiler = new ExpressionCompiler(new RexBuilder(join.getCluster().getTypeFactory())); + String expression = compiler.getExpression(remaining, join.getRowType(), join.getRowType()); + + Map expMap = new HashMap<>(); + for (Pair pair : Pair.zip(join.getRowType().getFieldList(), + join.getRowType().getFieldList())) { + String leftName = OperatorUtils.getFieldName(pair.left); + String rightName = OperatorUtils.getFieldName(pair.right); + expMap.put(leftName, rightName); + } + operator.setExpressionMap(expMap); + operator.setCondition(expression); + + String streamName = OperatorUtils.getUniqueStreamName(join.getRelTypeName() + "_Join", join.getRelTypeName() + + "_Filter"); + Class schema = TupleSchemaRegistry.getSchemaForRelDataType(context.schemaRegistry, streamName, + relInfo.getOutRelDataType()); + context.dag.setOutputPortAttribute(relInfo.getOutPort(), Context.PortContext.TUPLE_CLASS, schema); + context.dag.setInputPortAttribute(operator.input, Context.PortContext.TUPLE_CLASS, schema); + context.dag.addStream(streamName, relInfo.getOutPort(), operator.input); + + return new RelInfo("Join", relInfo.getInputPorts(), operator, operator.output, join.getRowType()); + } + + private RelInfo addInnerJoinOperator(Join join, List leftKeys, List rightKeys, RelContext context) + { + String leftKeyExpression = null; + String rightKeyExpression = null; + for (Integer leftKey : leftKeys) { + String name = OperatorUtils.getValidFieldName(join.getLeft().getRowType().getFieldList().get(leftKey)); + leftKeyExpression = (leftKeyExpression == null) ? name : leftKeyExpression + " + " + name; + } + + for (Integer rightKey : rightKeys) { + String name = OperatorUtils.getValidFieldName(join.getRight().getRowType().getFieldList().get(rightKey)); + rightKeyExpression = (rightKeyExpression == null) ? name : rightKeyExpression + " + " + name; + } + + String includeFieldStr = ""; + boolean first = true; + for (RelDataTypeField field : join.getLeft().getRowType().getFieldList()) { + if (first) { + first = false; + } else { + includeFieldStr += ","; + } + includeFieldStr += OperatorUtils.getValidFieldName(field); + } + includeFieldStr += ";"; + first = true; + for (RelDataTypeField field : join.getRight().getRowType().getFieldList()) { + if (first) { + first = false; + } else { + includeFieldStr += ","; + } + includeFieldStr += OperatorUtils.getValidFieldName(field); + } + + InnerJoinOperator innerJoin = context.dag.addOperator(OperatorUtils.getUniqueOperatorName(join.getRelTypeName()), + InnerJoinOperator.class); + innerJoin.setExpiryTime(1L); + // Number of buckets is set to 47000 because this is rounded number closer to sqrt of MAXINT. This guarantees + // even distribution of keys across buckets. + innerJoin.setNoOfBuckets(47000); + innerJoin.setTimeFieldsStr(""); + + innerJoin.setLeftKeyExpression(leftKeyExpression); + innerJoin.setRightKeyExpression(rightKeyExpression); + + innerJoin.setIncludeFieldStr(includeFieldStr); + + return new RelInfo("Join", Lists.newArrayList(innerJoin.input1, innerJoin.input2), innerJoin, + innerJoin.outputPort, join.getRowType()); + } + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/planner/RelInfo.java b/sql/src/main/java/org/apache/apex/malhar/sql/planner/RelInfo.java new file mode 100644 index 0000000000..29948ecebb --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/planner/RelInfo.java @@ -0,0 +1,121 @@ +/** + * 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.apex.malhar.sql.planner; + +import java.util.List; + +import org.apache.calcite.rel.type.RelDataType; +import org.apache.hadoop.classification.InterfaceStability; + +import com.datatorrent.api.Operator; + +/** + * This object communicates stream and connection data between various stages of relational algebra. + */ +@InterfaceStability.Evolving +public class RelInfo +{ + private List inputPorts; + private Operator operator; + private Operator.OutputPort outPort; + private RelDataType outRelDataType; + private Class clazz; + private String relName; + + public RelInfo(String relName, List inputPorts, Operator operator, Operator.OutputPort outPort, + RelDataType outRelDataType) + { + this.inputPorts = inputPorts; + this.relName = relName; + this.operator = operator; + this.outPort = outPort; + this.outRelDataType = outRelDataType; + this.clazz = null; + } + + public RelInfo(String relName, List inputPorts, Operator operator, Operator.OutputPort outPort, Class clazz) + { + this.inputPorts = inputPorts; + this.operator = operator; + this.outPort = outPort; + this.clazz = clazz; + this.relName = relName; + this.outRelDataType = null; + } + + public Class getClazz() + { + return clazz; + } + + public void setClazz(Class clazz) + { + this.clazz = clazz; + } + + public List getInputPorts() + { + return inputPorts; + } + + public void setInputPorts(List inputPorts) + { + this.inputPorts = inputPorts; + } + + public String getRelName() + { + return relName; + } + + public void setRelName(String relName) + { + this.relName = relName; + } + + public Operator getOperator() + { + return operator; + } + + public void setOperator(Operator operator) + { + this.operator = operator; + } + + public Operator.OutputPort getOutPort() + { + return outPort; + } + + public void setOutPort(Operator.OutputPort outPort) + { + this.outPort = outPort; + } + + public RelDataType getOutRelDataType() + { + return outRelDataType; + } + + public void setOutRelDataType(RelDataType outRelDataType) + { + this.outRelDataType = outRelDataType; + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/planner/RelNodeVisitor.java b/sql/src/main/java/org/apache/apex/malhar/sql/planner/RelNodeVisitor.java new file mode 100644 index 0000000000..68343ce6cc --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/planner/RelNodeVisitor.java @@ -0,0 +1,112 @@ +/** + * 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.apex.malhar.sql.planner; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.apex.malhar.sql.operators.OperatorUtils; +import org.apache.apex.malhar.sql.schema.TupleSchemaRegistry; + +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.rel.RelNode; + +import org.apache.hadoop.classification.InterfaceStability; + +import com.datatorrent.api.Context; +import com.datatorrent.api.DAG; +import com.datatorrent.api.Operator; + +/** + * This class is the main class that converts relational algebra to a sub-DAG. + */ +@InterfaceStability.Evolving +public class RelNodeVisitor +{ + private final DAG dag; + private final TupleSchemaRegistry tupleSchemaRegistry; + private final JavaTypeFactory typeFactory; + + public RelNodeVisitor(DAG dag, JavaTypeFactory typeFactory) + { + this.dag = dag; + this.typeFactory = typeFactory; + this.tupleSchemaRegistry = new TupleSchemaRegistry(); + } + + /** + * This is the main method in this relational node visitor which traverses the relational algebra in reverse direction + * and populate the given underlying DAG object. + * + * @param relNode RelNode which needs to be traversed. + * + * @return RelInfo representing information of current stage + * @throws Exception + */ + public final RelInfo traverse(RelNode relNode) throws Exception + { + List inputStreams = new ArrayList<>(); + for (RelNode input : relNode.getInputs()) { + inputStreams.add(traverse(input)); + } + + ApexRelNode.RelContext relContext = new ApexRelNode.RelContext(dag, typeFactory, tupleSchemaRegistry); + + RelInfo currentNodeRelInfo; + ApexRelNode apexRelNode = ApexRelNode.relNodeMapping.get(relNode.getClass()); + if (apexRelNode == null) { + throw new UnsupportedOperationException("RelNode " + relNode.getRelTypeName() + " is not supported."); + } + currentNodeRelInfo = apexRelNode.visit(relContext, relNode, inputStreams); + + if (currentNodeRelInfo != null && inputStreams.size() != 0) { + for (int i = 0; i < inputStreams.size(); i++) { + RelInfo inputStream = inputStreams.get(i); + Operator.OutputPort outputPort = inputStream.getOutPort(); + Operator.InputPort inputPort = currentNodeRelInfo.getInputPorts().get(i); + + String streamName = OperatorUtils.getUniqueStreamName(inputStream.getRelName(), + currentNodeRelInfo.getRelName()); + Class schema; + if (inputStream.getOutRelDataType() != null) { + schema = TupleSchemaRegistry.getSchemaForRelDataType(tupleSchemaRegistry, streamName, + inputStream.getOutRelDataType()); + } else if (inputStream.getClazz() != null) { + schema = inputStream.getClazz(); + } else { + throw new RuntimeException("Unexpected condition reached."); + } + dag.setOutputPortAttribute(outputPort, Context.PortContext.TUPLE_CLASS, schema); + dag.setInputPortAttribute(inputPort, Context.PortContext.TUPLE_CLASS, schema); + dag.addStream(streamName, outputPort, inputPort); + } + } + + if (currentNodeRelInfo.getOutPort() == null) { + // End of the pipeline. + String schemaJar = tupleSchemaRegistry.generateCommonJar(); + + String jars = dag.getAttributes().get(Context.DAGContext.LIBRARY_JARS); + dag.setAttribute(Context.DAGContext.LIBRARY_JARS, + ((jars != null) && (jars.length() != 0)) ? jars + "," + schemaJar : schemaJar); + } + + return currentNodeRelInfo; + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/schema/ApexSQLTable.java b/sql/src/main/java/org/apache/apex/malhar/sql/schema/ApexSQLTable.java new file mode 100644 index 0000000000..6d16f634fc --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/schema/ApexSQLTable.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.apex.malhar.sql.schema; + +import java.util.Map; + +import org.apache.apex.malhar.sql.table.Endpoint; +import org.apache.calcite.DataContext; +import org.apache.calcite.linq4j.Enumerable; +import org.apache.calcite.rel.RelCollations; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.schema.ScannableTable; +import org.apache.calcite.schema.Schema; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.schema.Statistic; +import org.apache.calcite.schema.Statistics; +import org.apache.calcite.schema.StreamableTable; +import org.apache.calcite.schema.Table; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.hadoop.classification.InterfaceStability; + +import com.google.common.collect.ImmutableList; + +/** + * This is representation of Apex source/destination to Calcite's {@link StreamableTable} table. + * Any table that gets registered with {@link org.apache.apex.malhar.sql.SQLExecEnvironment} + * gets registered as {@link ApexSQLTable}. + */ +@InterfaceStability.Evolving +public class ApexSQLTable implements ScannableTable, StreamableTable +{ + private SchemaPlus schema; + private String name; + private Map operands; + private RelDataType rowType; + private Endpoint endpoint; + + public ApexSQLTable(SchemaPlus schemaPlus, String name, Map operands, RelDataType rowType, + Endpoint endpoint) + { + this.schema = schemaPlus; + this.name = name; + this.operands = operands; + this.rowType = rowType; + this.endpoint = endpoint; + } + + public ApexSQLTable(SchemaPlus schema, String name, Endpoint endpoint) + { + this(schema, name, null, null, endpoint); + } + + @Override + public Enumerable scan(DataContext dataContext) + { + return null; + } + + @Override + public Table stream() + { + return new ApexSQLTable(schema, name, operands, rowType, endpoint); + } + + @Override + public RelDataType getRowType(RelDataTypeFactory relDataTypeFactory) + { + if (rowType == null) { + rowType = endpoint.getRowType(relDataTypeFactory); + } + return rowType; + } + + @Override + public Statistic getStatistic() + { + return Statistics.of(100d, ImmutableList.of(), RelCollations.createSingleton(0)); + } + + @Override + public Schema.TableType getJdbcTableType() + { + return Schema.TableType.STREAM; + } + + public SchemaPlus getSchema() + { + return schema; + } + + public String getName() + { + return name; + } + + public Map getOperands() + { + return operands; + } + + public RelDataType getRowType() + { + return rowType; + } + + public Endpoint getEndpoint() + { + return endpoint; + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/schema/ApexSQLTableFactory.java b/sql/src/main/java/org/apache/apex/malhar/sql/schema/ApexSQLTableFactory.java new file mode 100644 index 0000000000..c18f8545d1 --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/schema/ApexSQLTableFactory.java @@ -0,0 +1,66 @@ +/** + * 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.apex.malhar.sql.schema; + +import java.util.Map; + +import org.apache.apex.malhar.sql.table.CSVMessageFormat; +import org.apache.apex.malhar.sql.table.Endpoint; +import org.apache.apex.malhar.sql.table.FileEndpoint; +import org.apache.apex.malhar.sql.table.KafkaEndpoint; +import org.apache.apex.malhar.sql.table.MessageFormat; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.schema.Table; +import org.apache.calcite.schema.TableFactory; +import org.apache.hadoop.classification.InterfaceStability; + +@InterfaceStability.Evolving +public class ApexSQLTableFactory implements TableFactory +{ + @SuppressWarnings("unchecked") + @Override + public Table create(SchemaPlus schemaPlus, String name, Map operands, RelDataType rowType) + { + Endpoint endpoint; + String endpointSystemType = (String)operands.get(Endpoint.ENDPOINT); + + if (endpointSystemType.equalsIgnoreCase(Endpoint.EndpointType.FILE.name())) { + endpoint = new FileEndpoint(); + } else if (endpointSystemType.equalsIgnoreCase(Endpoint.EndpointType.KAFKA.name())) { + endpoint = new KafkaEndpoint(); + } else { + throw new RuntimeException("Cannot find endpoint"); + } + endpoint.setEndpointOperands((Map)operands.get(Endpoint.SYSTEM_OPERANDS)); + + MessageFormat mf; + String messageFormat = (String)operands.get(MessageFormat.MESSAGE_FORMAT); + if (messageFormat.equalsIgnoreCase(MessageFormat.MessageFormatType.CSV.name())) { + mf = new CSVMessageFormat(); + } else { + throw new RuntimeException("Cannot find message format"); + } + mf.setMessageFormatOperands((Map)operands.get(MessageFormat.MESSAGE_FORMAT_OPERANDS)); + + endpoint.setMessageFormat(mf); + + return new ApexSQLTable(schemaPlus, name, operands, rowType, endpoint); + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/schema/TupleSchemaRegistry.java b/sql/src/main/java/org/apache/apex/malhar/sql/schema/TupleSchemaRegistry.java new file mode 100644 index 0000000000..79242982fb --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/schema/TupleSchemaRegistry.java @@ -0,0 +1,227 @@ +/** + * 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.apex.malhar.sql.schema; + +import java.io.File; +import java.io.IOException; +import java.sql.Time; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.jar.JarOutputStream; +import java.util.zip.ZipEntry; + +import org.codehaus.jettison.json.JSONException; + +import org.apache.apex.malhar.lib.utils.ClassLoaderUtils; +import org.apache.apex.malhar.sql.codegen.BeanClassGenerator; +import org.apache.apex.malhar.sql.operators.OperatorUtils; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; + +import org.apache.commons.lang.ClassUtils; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +@InterfaceStability.Evolving +public class TupleSchemaRegistry +{ + public static final String FQCN_PACKAGE = "org.apache.apex.generated.schema."; + private Map schemas = new HashMap<>(); + + public Schema createNewSchema(String name) + { + if (schemas.containsKey(name)) { + return schemas.get(name); + } + + Schema schema = new Schema(); + schema.name = name; + schemas.put(name, schema); + + return schema; + } + + public Schema getSchemaDefinition(String name) + { + return schemas.get(name); + } + + public String generateCommonJar() throws IOException + { + File file = File.createTempFile("schemaSQL", ".jar"); + + FileSystem fs = FileSystem.newInstance(file.toURI(), new Configuration()); + FSDataOutputStream out = fs.create(new Path(file.getAbsolutePath())); + JarOutputStream jout = new JarOutputStream(out); + + for (Schema schema : schemas.values()) { + jout.putNextEntry(new ZipEntry(schema.fqcn.replace(".", "/") + ".class")); + jout.write(schema.beanClassBytes); + jout.closeEntry(); + } + + jout.close(); + out.close(); + + return file.getAbsolutePath(); + } + + public static Class getSchemaForRelDataType(TupleSchemaRegistry registry, String schemaName, RelDataType rowType) + { + if (rowType.isStruct()) { + TupleSchemaRegistry.Schema newSchema = registry.createNewSchema(schemaName); + for (RelDataTypeField field : rowType.getFieldList()) { + RelDataType type = field.getType(); + newSchema.addField(OperatorUtils.getValidFieldName(field), convertPrimitiveToSqlType(type)); + } + try { + newSchema.generateBean(); + } catch (IOException | JSONException e) { + throw new RuntimeException("Failed to generate schema", e); + } + return newSchema.beanClass; + } else { + throw new UnsupportedOperationException("Non-struct row type is not implemented."); + } + } + + private static Class convertPrimitiveToSqlType(RelDataType type) + { + /* I hope that following this method instead of calling value.value() is better + because we can catch any type mismatches. */ + switch (type.getSqlTypeName()) { + case BOOLEAN: + return Boolean.class; + case TINYINT: + case SMALLINT: + case INTEGER: + return Integer.class; + case BIGINT: + return Long.class; + case REAL: + return Float.class; + case FLOAT: + case DOUBLE: + return Double.class; + case DATE: + return Date.class; + case TIME: + return Date.class; + case TIMESTAMP: + return Date.class; + case CHAR: + case VARCHAR: + return String.class; + case BINARY: + case VARBINARY: + return Byte.class; + case ANY: + case SYMBOL: + return Object.class; + default: + throw new RuntimeException(String.format("Unsupported type %s", type.getSqlTypeName())); + } + } + + public enum Type + { + BOOLEAN(Boolean.class), SHORT(Short.class), INTEGER(Integer.class), LONG(Long.class), + FLOAT(Float.class), DOUBLE(Double.class), STRING(String.class), OBJECT(Object.class), + DATE(Date.class), TIME(Time.class); + + private Class javaType; + + Type(Class javaType) + { + this.javaType = javaType; + } + + public static Type getFromJavaType(Class type) + { + for (Type supportType : Type.values()) { + if (supportType.getJavaType() == ClassUtils.primitiveToWrapper(type)) { + return supportType; + } + } + + return OBJECT; + } + + public Class getJavaType() + { + return javaType; + } + } + + public static class Schema + { + public String name; + public String fqcn; + public List fieldList = new ArrayList<>(); + public Class beanClass; + public byte[] beanClassBytes; + + public Schema addField(String fieldName, Class fieldType) + { + fieldList.add(new SQLFieldInfo(fieldName, Type.getFromJavaType(fieldType))); + return this; + } + + public Schema generateBean() throws IOException, JSONException + { + // Generate + this.fqcn = FQCN_PACKAGE + name; + + // Use Bean Class generator to generate the class + this.beanClassBytes = BeanClassGenerator.createAndWriteBeanClass(this.fqcn, fieldList); + this.beanClass = ClassLoaderUtils.readBeanClass(fqcn, beanClassBytes); + + return this; + } + } + + public static class SQLFieldInfo + { + String columnName; + Type type; + + public SQLFieldInfo(String columnName, Type type) + { + this.columnName = columnName; + this.type = type; + } + + public String getColumnName() + { + return columnName; + } + + public Type getType() + { + return type; + } + } + +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/table/CSVMessageFormat.java b/sql/src/main/java/org/apache/apex/malhar/sql/table/CSVMessageFormat.java new file mode 100644 index 0000000000..a96df65142 --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/table/CSVMessageFormat.java @@ -0,0 +1,138 @@ +/** + * 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.apex.malhar.sql.table; + +import java.util.Map; + +import org.apache.apex.malhar.sql.operators.OperatorUtils; +import org.apache.apex.malhar.sql.planner.RelInfo; +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.hadoop.classification.InterfaceStability; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; + +import com.datatorrent.api.DAG; +import com.datatorrent.api.Operator; +import com.datatorrent.contrib.formatter.CsvFormatter; +import com.datatorrent.contrib.parser.CsvParser; +import com.datatorrent.contrib.parser.DelimitedSchema; + +@InterfaceStability.Evolving +public class CSVMessageFormat implements MessageFormat +{ + public static final String CSV_SCHEMA = "schema"; + private Map operands; + + public CSVMessageFormat() + { + } + + public CSVMessageFormat(String schema) + { + this.operands = ImmutableMap.of(CSV_SCHEMA, schema); + } + + @Override + public MessageFormatType getMessageFormatType() + { + return MessageFormatType.CSV; + } + + @Override + public void setMessageFormatOperands(Map operands) + { + this.operands = operands; + } + + @Override + public RelInfo populateInputDAG(DAG dag, JavaTypeFactory typeFactory) + { + CsvParser csvParser = dag.addOperator(OperatorUtils.getUniqueOperatorName("CSVParser"), CsvParser.class); + csvParser.setSchema((String)operands.get(CSV_SCHEMA)); + + return new RelInfo("CSVParser", Lists.newArrayList(csvParser.in), csvParser, csvParser.out, + getRowType(typeFactory)); + } + + @Override + public RelInfo populateOutputDAG(DAG dag, JavaTypeFactory typeFactory) + { + CsvFormatter formatter = dag.addOperator(OperatorUtils.getUniqueOperatorName("CSVFormatter"), CsvFormatter.class); + formatter.setSchema((String)operands.get(CSV_SCHEMA)); + + return new RelInfo("CSVFormatter", Lists.newArrayList(formatter.in), formatter, formatter.out, + getRowType(typeFactory)); + } + + @Override + public RelDataType getRowType(RelDataTypeFactory typeFactory) + { + String schema = (String)operands.get(CSV_SCHEMA); + RelDataTypeFactory.FieldInfoBuilder builder = typeFactory.builder(); + + DelimitedSchema delimitedSchema = new DelimitedSchema(schema); + for (DelimitedSchema.Field field : delimitedSchema.getFields()) { + builder.add(field.getName(), convertField(typeFactory, field.getType())); + } + + return builder.build(); + } + + private RelDataType convertField(RelDataTypeFactory typeFactory, DelimitedSchema.FieldType type) + { + RelDataType relDataType; + switch (type) { + case BOOLEAN: + relDataType = typeFactory.createSqlType(SqlTypeName.BOOLEAN); + break; + case DOUBLE: + relDataType = typeFactory.createSqlType(SqlTypeName.DOUBLE); + break; + case INTEGER: + relDataType = typeFactory.createSqlType(SqlTypeName.INTEGER); + break; + case FLOAT: + relDataType = typeFactory.createSqlType(SqlTypeName.FLOAT); + break; + case LONG: + relDataType = typeFactory.createSqlType(SqlTypeName.BIGINT); + break; + case SHORT: + relDataType = typeFactory.createSqlType(SqlTypeName.SMALLINT); + break; + case CHARACTER: + relDataType = typeFactory.createSqlType(SqlTypeName.CHAR); + break; + case STRING: + relDataType = typeFactory.createSqlType(SqlTypeName.VARCHAR); + break; + case DATE: + relDataType = typeFactory.createSqlType(SqlTypeName.TIMESTAMP); + break; + default: + relDataType = typeFactory.createSqlType(SqlTypeName.ANY); + } + + return relDataType; + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/table/Endpoint.java b/sql/src/main/java/org/apache/apex/malhar/sql/table/Endpoint.java new file mode 100644 index 0000000000..41a26de9a7 --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/table/Endpoint.java @@ -0,0 +1,100 @@ +/** + * 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.apex.malhar.sql.table; + +import java.util.Map; + +import org.apache.apex.malhar.sql.planner.RelInfo; +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.hadoop.classification.InterfaceStability; + +import com.datatorrent.api.DAG; + +/** + * This interface defines abstract table and how it should be operated with. + * Endpoint interface can be implemented for any type of data source eg. Kafka, File, JDBC etc. + * Implementation of Endpoint interface should define how the table should represented for both input OR output side. + */ +@InterfaceStability.Evolving +public interface Endpoint +{ + String ENDPOINT = "endpoint"; + String SYSTEM_OPERANDS = "endpointOperands"; + + /** + * Returns target type system + * @return Returns target type system + */ + EndpointType getTargetType(); + + /** + * Set Endpoint operands. This method is used when the table definitions are provided using calcite schema format. + * This is the map which is present against key "endpointOperands" in calcite schema definition input file. + * + * @param operands Map of endpoint operands. + */ + void setEndpointOperands(Map operands); + + /** + * Message Format type which defines how the data should be interpreted for both input and output side. + * + * @param messageFormat Object of type MessageFormat + */ + void setMessageFormat(MessageFormat messageFormat); + + /** + * Implementation of this method should populate Apex DAG if this table is at input side of pipeline. + * + * @param dag {@link DAG} object to be populated + * @param typeFactory Java Type Factory + * + * @return Returns {@link RelInfo} describing output of this input phase. + */ + RelInfo populateInputDAG(DAG dag, JavaTypeFactory typeFactory); + + /** + * Implementation of this method should populate Apex DAG if table is at output side of pipeline. + * + * @param dag {@link DAG} object to be populated + * @param typeFactory Java Type Factory + * @return Returns {@link RelInfo} describing expected input of this output phase. + */ + RelInfo populateOutputDAG(DAG dag, JavaTypeFactory typeFactory); + + /** + * This method returns what should be the input data type to output phase OR output data type of input phase. + * + * @param typeFactory Java Type Factory for data type conversions. + * + * @return {@link RelDataType} representing data type format. + */ + RelDataType getRowType(RelDataTypeFactory typeFactory); + + /** + * Type of Endpoints + */ + enum EndpointType + { + FILE, + KAFKA, + PORT + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/table/FileEndpoint.java b/sql/src/main/java/org/apache/apex/malhar/sql/table/FileEndpoint.java new file mode 100644 index 0000000000..cac32a4c95 --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/table/FileEndpoint.java @@ -0,0 +1,119 @@ +/** + * 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.apex.malhar.sql.table; + +import java.util.Map; + +import org.apache.apex.malhar.lib.fs.GenericFileOutputOperator; +import org.apache.apex.malhar.sql.operators.LineReader; +import org.apache.apex.malhar.sql.operators.OperatorUtils; +import org.apache.apex.malhar.sql.planner.RelInfo; +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.hadoop.classification.InterfaceStability; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; + +import com.datatorrent.api.DAG; +import com.datatorrent.api.Operator; + +/** + * This is an implementation of {@link Endpoint} which defined how data should be read/written to file system. + */ +@InterfaceStability.Evolving +public class FileEndpoint implements Endpoint +{ + public static final String FILE_INPUT_DIRECTORY = "directory"; + public static final String FILE_OUT_PATH = "outputFilePath"; + public static final String FILE_OUT_NAME = "outputFileName"; + + private MessageFormat messageFormat; + + private Map operands; + + public FileEndpoint() + { + } + + public FileEndpoint(String directory, MessageFormat messageFormat) + { + this.messageFormat = messageFormat; + this.operands = ImmutableMap.of(FILE_INPUT_DIRECTORY, directory); + } + + public FileEndpoint(String directory, String fileName, MessageFormat messageFormat) + { + this.messageFormat = messageFormat; + this.operands = ImmutableMap.of(FILE_OUT_PATH, directory, FILE_OUT_NAME, fileName); + } + + @Override + public EndpointType getTargetType() + { + return EndpointType.FILE; + } + + @Override + public void setEndpointOperands(Map operands) + { + this.operands = operands; + } + + @Override + public void setMessageFormat(MessageFormat messageFormat) + { + this.messageFormat = messageFormat; + } + + @Override + public RelInfo populateInputDAG(DAG dag, JavaTypeFactory typeFactory) + { + LineReader fileInput = dag.addOperator(OperatorUtils.getUniqueOperatorName("FileInput"), LineReader.class); + fileInput.setDirectory((String)operands.get(FILE_INPUT_DIRECTORY)); + + RelInfo spec = messageFormat.populateInputDAG(dag, typeFactory); + dag.addStream(OperatorUtils.getUniqueStreamName("File", "Parser"), fileInput.output, spec.getInputPorts().get(0)); + return new RelInfo("Input", Lists.newArrayList(), spec.getOperator(), spec.getOutPort(), + messageFormat.getRowType(typeFactory)); + } + + @Override + public RelInfo populateOutputDAG(DAG dag, JavaTypeFactory typeFactory) + { + RelInfo spec = messageFormat.populateOutputDAG(dag, typeFactory); + + GenericFileOutputOperator.StringFileOutputOperator fileOutput = + dag.addOperator(OperatorUtils.getUniqueOperatorName("FileOutput"), + GenericFileOutputOperator.StringFileOutputOperator.class); + fileOutput.setFilePath((String)operands.get(FILE_OUT_PATH)); + fileOutput.setOutputFileName((String)operands.get(FILE_OUT_NAME)); + + dag.addStream(OperatorUtils.getUniqueStreamName("Formatter", "File"), spec.getOutPort(), fileOutput.input); + + return new RelInfo("Output", spec.getInputPorts(), spec.getOperator(), null, messageFormat.getRowType(typeFactory)); + } + + @Override + public RelDataType getRowType(RelDataTypeFactory typeFactory) + { + return messageFormat.getRowType(typeFactory); + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/table/KafkaEndpoint.java b/sql/src/main/java/org/apache/apex/malhar/sql/table/KafkaEndpoint.java new file mode 100644 index 0000000000..56419c3776 --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/table/KafkaEndpoint.java @@ -0,0 +1,136 @@ +/** + * 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.apex.malhar.sql.table; + +import java.util.Map; +import java.util.Properties; + +import org.apache.apex.malhar.kafka.KafkaSinglePortInputOperator; +import org.apache.apex.malhar.kafka.KafkaSinglePortOutputOperator; +import org.apache.apex.malhar.sql.operators.OperatorUtils; +import org.apache.apex.malhar.sql.planner.RelInfo; +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; + +import com.datatorrent.api.DAG; +import com.datatorrent.api.Operator; + +/** + * This is an implementation of {@link Endpoint} which defined how data should be read/written from kafka messaging system + */ +@InterfaceStability.Evolving +public class KafkaEndpoint implements Endpoint +{ + public static final String KEY_DESERIALIZER = "org.apache.kafka.common.serialization.StringDeserializer"; + public static final String VALUE_DESERIALIZER = "org.apache.kafka.common.serialization.StringDeserializer"; + public static final String KEY_SERIALIZER = "org.apache.kafka.common.serialization.StringSerializer"; + public static final String VALUE_SERIALIZER = "org.apache.kafka.common.serialization.StringSerializer"; + + + public static final String KAFKA_SERVERS = "servers"; + public static final String KAFKA_TOPICS = "topics"; + + private MessageFormat messageFormat; + + private Map operands; + + public KafkaEndpoint() + { + } + + public KafkaEndpoint(String kafkaServers, String topics, MessageFormat messageFormat) + { + this.messageFormat = messageFormat; + this.operands = ImmutableMap.of(KAFKA_SERVERS, kafkaServers, KAFKA_TOPICS, topics); + } + + @Override + public EndpointType getTargetType() + { + return EndpointType.KAFKA; + } + + @Override + public void setEndpointOperands(Map operands) + { + this.operands = operands; + } + + @Override + public void setMessageFormat(MessageFormat messageFormat) + { + this.messageFormat = messageFormat; + } + + @Override + public RelInfo populateInputDAG(DAG dag, JavaTypeFactory typeFactory) + { + KafkaSinglePortInputOperator kafkaInput = dag.addOperator(OperatorUtils.getUniqueOperatorName("KafkaInput"), + KafkaSinglePortInputOperator.class); + kafkaInput.setTopics((String)operands.get(KAFKA_TOPICS)); + kafkaInput.setInitialOffset("EARLIEST"); + + Properties props = new Properties(); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, operands.get(KAFKA_SERVERS)); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, KEY_DESERIALIZER); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, VALUE_DESERIALIZER); + kafkaInput.setConsumerProps(props); + + kafkaInput.setClusters((String)operands.get(KAFKA_SERVERS)); + + RelInfo spec = messageFormat.populateInputDAG(dag, typeFactory); + dag.addStream(OperatorUtils.getUniqueStreamName("Kafka", "Parser"), kafkaInput.outputPort, + spec.getInputPorts().get(0)); + return new RelInfo("Input", Lists.newArrayList(), spec.getOperator(), spec.getOutPort(), + messageFormat.getRowType(typeFactory)); + } + + @Override + public RelInfo populateOutputDAG(DAG dag, JavaTypeFactory typeFactory) + { + RelInfo spec = messageFormat.populateOutputDAG(dag, typeFactory); + + KafkaSinglePortOutputOperator kafkaOutput = dag.addOperator(OperatorUtils.getUniqueOperatorName("KafkaOutput"), + KafkaSinglePortOutputOperator.class); + kafkaOutput.setTopic((String)operands.get(KAFKA_TOPICS)); + + Properties props = new Properties(); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, VALUE_SERIALIZER); + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, KEY_SERIALIZER); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, operands.get(KAFKA_SERVERS)); + kafkaOutput.setProperties(props); + + dag.addStream(OperatorUtils.getUniqueStreamName("Formatter", "Kafka"), spec.getOutPort(), kafkaOutput.inputPort); + + return new RelInfo("Output", spec.getInputPorts(), spec.getOperator(), null, messageFormat.getRowType(typeFactory)); + } + + @Override + public RelDataType getRowType(RelDataTypeFactory typeFactory) + { + return messageFormat.getRowType(typeFactory); + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/table/MessageFormat.java b/sql/src/main/java/org/apache/apex/malhar/sql/table/MessageFormat.java new file mode 100644 index 0000000000..80fef93c3f --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/table/MessageFormat.java @@ -0,0 +1,88 @@ +/** + * 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.apex.malhar.sql.table; + +import java.util.Map; + +import org.apache.apex.malhar.sql.planner.RelInfo; +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.hadoop.classification.InterfaceStability; + +import com.datatorrent.api.DAG; + +/** + * This interface defines how message should be parsed from input or formatted for output. + * The implementation of this interface should define both parsing and formatting representation for data. + */ +@InterfaceStability.Evolving +public interface MessageFormat +{ + String MESSAGE_FORMAT = "messageFormat"; + String MESSAGE_FORMAT_OPERANDS = "messageFormatOperands"; + + /** + * Gives type of {@link MessageFormat} + * @return Returns type of {@link MessageFormat} + */ + MessageFormatType getMessageFormatType(); + + /** + * Set messageFormat operands. This method is used when the table definitions are provided using calcite schema format. + * This is the map which is present against key "endpointOperands" in calcite schema definition input file. + * @param operands + */ + void setMessageFormatOperands(Map operands); + + /** + * Implementation of this method should populate the DAG for parsing logic for the data received from {@link Endpoint} + * + * @param dag {@link DAG} object to be populated + * @param typeFactory Java Type Factory + * @return Returns {@link RelInfo} defining output data type definition after parsing of data. + */ + RelInfo populateInputDAG(DAG dag, JavaTypeFactory typeFactory); + + /** + * Implementation of this method should populate the DAG for formatting logic of data to be written to {@link Endpoint} + * + * @param dag {@link DAG} object to be populated + * @param typeFactory Java Type Factory + * @return Returns {@link RelInfo} defining expected input for formatting ot data. + */ + RelInfo populateOutputDAG(DAG dag, JavaTypeFactory typeFactory); + + /** + * This method returns what should be the input data type to output phase OR output data type of input phase. + * + * @param typeFactory Java Type Factory for data type conversions. + * + * @return {@link RelDataType} representing data type format. + */ + RelDataType getRowType(RelDataTypeFactory typeFactory); + + /** + * Message Format types + */ + enum MessageFormatType + { + CSV + } +} diff --git a/sql/src/main/java/org/apache/apex/malhar/sql/table/StreamEndpoint.java b/sql/src/main/java/org/apache/apex/malhar/sql/table/StreamEndpoint.java new file mode 100644 index 0000000000..5462e429e6 --- /dev/null +++ b/sql/src/main/java/org/apache/apex/malhar/sql/table/StreamEndpoint.java @@ -0,0 +1,147 @@ +/** + * 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.apex.malhar.sql.table; + +import java.lang.reflect.Field; +import java.util.Date; +import java.util.Map; + +import org.apache.apex.malhar.sql.planner.RelInfo; + +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.hadoop.classification.InterfaceStability; + +import com.google.common.collect.Lists; + +import com.datatorrent.api.DAG; +import com.datatorrent.api.Operator; + +/** + * This is an implementation of {@link Endpoint} which defined how data should be read/written to a Apex streaming port. + */ +@InterfaceStability.Evolving +public class StreamEndpoint implements Endpoint +{ + private Operator.InputPort inputPort; + private Operator.OutputPort outputPort; + private Class pojoClass; + private Map fieldMapping; + + public StreamEndpoint(Operator.InputPort port, Class pojoClass) + { + this.inputPort = port; + this.pojoClass = pojoClass; + } + + public StreamEndpoint(Operator.InputPort port, Map fieldMapping) + { + this.inputPort = port; + this.fieldMapping = fieldMapping; + } + + public StreamEndpoint(Operator.OutputPort outputPort, Class pojoClass) + { + this.outputPort = outputPort; + this.pojoClass = pojoClass; + } + + public StreamEndpoint(Operator.OutputPort port, Map fieldMapping) + { + this.outputPort = port; + this.fieldMapping = fieldMapping; + } + + @Override + public EndpointType getTargetType() + { + return EndpointType.PORT; + } + + @Override + public void setEndpointOperands(Map operands) + { + } + + @Override + public void setMessageFormat(MessageFormat messageFormat) + { + } + + @Override + public RelInfo populateInputDAG(DAG dag, JavaTypeFactory typeFactory) + { + return new RelInfo("StreamInput", Lists.newArrayList(), null, outputPort, getRowType(typeFactory)); + } + + @Override + public RelInfo populateOutputDAG(DAG dag, JavaTypeFactory typeFactory) + { + return new RelInfo("StreamOutput", Lists.newArrayList(inputPort), null, null, getRowType(typeFactory)); + } + + @Override + public RelDataType getRowType(RelDataTypeFactory typeFactory) + { + RelDataTypeFactory.FieldInfoBuilder builder = typeFactory.builder(); + if (fieldMapping != null) { + for (Map.Entry entry : fieldMapping.entrySet()) { + builder.add(entry.getKey(), convertField(typeFactory, entry.getValue())); + } + } else if (pojoClass != null) { + for (Field field : pojoClass.getDeclaredFields()) { + builder.add(field.getName(), convertField(typeFactory, field.getType())); + } + } else { + throw new RuntimeException("Either fieldMapping or pojoClass needs to be set."); + } + + return builder.build(); + } + + private RelDataType convertField(RelDataTypeFactory typeFactory, Class type) + { + RelDataType relDataType; + + if ((type == Boolean.class) || (type == boolean.class)) { + relDataType = typeFactory.createSqlType(SqlTypeName.BOOLEAN); + } else if ((type == Double.class) || (type == double.class)) { + relDataType = typeFactory.createSqlType(SqlTypeName.DOUBLE); + } else if ((type == Integer.class) || (type == int.class)) { + relDataType = typeFactory.createSqlType(SqlTypeName.INTEGER); + } else if ((type == Float.class) || (type == float.class)) { + relDataType = typeFactory.createSqlType(SqlTypeName.FLOAT); + } else if ((type == Long.class) || (type == long.class)) { + relDataType = typeFactory.createSqlType(SqlTypeName.BIGINT); + } else if ((type == Short.class) || (type == short.class)) { + relDataType = typeFactory.createSqlType(SqlTypeName.SMALLINT); + } else if ((type == Character.class) || (type == char.class) || (type == Byte.class) || (type == byte.class)) { + relDataType = typeFactory.createSqlType(SqlTypeName.CHAR); + } else if (type == String.class) { + relDataType = typeFactory.createSqlType(SqlTypeName.VARCHAR); + } else if (type == Date.class) { + relDataType = typeFactory.createSqlType(SqlTypeName.TIMESTAMP); + } else { + relDataType = typeFactory.createSqlType(SqlTypeName.ANY); + } + return relDataType; + } +} diff --git a/sql/src/test/java/org/apache/apex/malhar/sql/FileEndpointTest.java b/sql/src/test/java/org/apache/apex/malhar/sql/FileEndpointTest.java new file mode 100644 index 0000000000..900fd1020e --- /dev/null +++ b/sql/src/test/java/org/apache/apex/malhar/sql/FileEndpointTest.java @@ -0,0 +1,249 @@ +/** + * 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.apex.malhar.sql; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.PrintStream; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.TimeZone; + +import javax.validation.ConstraintViolationException; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; + +import org.apache.apex.malhar.sql.table.CSVMessageFormat; +import org.apache.apex.malhar.sql.table.FileEndpoint; + +import org.apache.commons.io.FileUtils; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import com.google.common.base.Predicates; +import com.google.common.collect.Collections2; +import com.google.common.collect.Lists; + +import com.datatorrent.api.DAG; +import com.datatorrent.api.LocalMode; +import com.datatorrent.api.StreamingApplication; + +public class FileEndpointTest +{ + private TimeZone defaultTZ; + private static String outputFolder = "target/output/"; + + @Rule + public TestName testName = new TestName(); + + public static String apex_concat_str(String s1, String s2) + { + return s1 + s2; + } + + @Before + public void setUp() throws Exception + { + defaultTZ = TimeZone.getDefault(); + TimeZone.setDefault(TimeZone.getTimeZone("GMT")); + + outputFolder += testName.getMethodName() + "/"; + } + + @After + public void tearDown() throws Exception + { + TimeZone.setDefault(defaultTZ); + } + + @Test + public void testApplication() throws Exception + { + File modelFile = new File("src/test/resources/model/model_file_csv.json"); + String model = FileUtils.readFileToString(modelFile); + + PrintStream originalSysout = System.out; + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + try { + LocalMode lma = LocalMode.newInstance(); + Configuration conf = new Configuration(false); + lma.prepareDAG(new Application(model), conf); + LocalMode.Controller lc = lma.getController(); + lc.runAsync(); + + waitTillStdoutIsPopulated(baos, 30000); + + lc.shutdown(); + } catch (ConstraintViolationException e) { + Assert.fail("constraint violations: " + e.getConstraintViolations()); + } catch (Exception e) { + Assert.fail("Exception: " + e); + } + + System.setOut(originalSysout); + + String[] sout = baos.toString().split(System.lineSeparator()); + Collection filter = Collections2.filter(Arrays.asList(sout), Predicates.containsPattern("Delta Record:")); + + String[] actualLines = filter.toArray(new String[filter.size()]); + Assert.assertEquals(6, actualLines.length); + Assert.assertTrue(actualLines[0].contains("RowTime=Mon Feb 15 10:15:00 GMT 2016, Product=paint1")); + Assert.assertTrue(actualLines[1].contains("RowTime=Mon Feb 15 10:16:00 GMT 2016, Product=paint2")); + Assert.assertTrue(actualLines[2].contains("RowTime=Mon Feb 15 10:17:00 GMT 2016, Product=paint3")); + Assert.assertTrue(actualLines[3].contains("RowTime=Mon Feb 15 10:18:00 GMT 2016, Product=paint4")); + Assert.assertTrue(actualLines[4].contains("RowTime=Mon Feb 15 10:19:00 GMT 2016, Product=paint5")); + Assert.assertTrue(actualLines[5].contains("RowTime=Mon Feb 15 10:10:00 GMT 2016, Product=abcde6")); + } + + private boolean waitTillStdoutIsPopulated(ByteArrayOutputStream baos, int timeout) throws InterruptedException, + IOException + { + long now = System.currentTimeMillis(); + Collection filter = Lists.newArrayList(); + while (System.currentTimeMillis() - now < timeout) { + baos.flush(); + String[] sout = baos.toString().split(System.lineSeparator()); + filter = Collections2.filter(Arrays.asList(sout), Predicates.containsPattern("Delta Record:")); + if (filter.size() != 0) { + break; + } + + Thread.sleep(500); + } + + return (filter.size() != 0); + } + + @Test + public void testApplicationSelectInsertWithAPI() throws Exception + { + try { + LocalMode lma = LocalMode.newInstance(); + Configuration conf = new Configuration(false); + lma.prepareDAG(new ApplicationSelectInsertWithAPI(), conf); + + LocalMode.Controller lc = lma.getController(); + lc.runAsync(); + /** + * Wait time is 40 sec to ensure that checkpoint happens. AbstractFileOutputOperators flushes the stream + * in beforeCheckpoint call. + */ + Assert.assertTrue(waitTillFileIsPopulated(outputFolder, 40000)); + lc.shutdown(); + } catch (Exception e) { + Assert.fail("constraint violations: " + e); + } + + File file = new File(outputFolder); + File file1 = new File(outputFolder + file.list()[0]); + List strings = FileUtils.readLines(file1); + + String[] actualLines = strings.toArray(new String[strings.size()]); + + String[] expectedLines = new String[]{"15/02/2016 10:18:00 +0000,15/02/2016 00:00:00 +0000,OILPAINT4", "", + "15/02/2016 10:19:00 +0000,15/02/2016 00:00:00 +0000,OILPAINT5", ""}; + Assert.assertTrue(Arrays.deepEquals(actualLines, expectedLines)); + } + + private boolean waitTillFileIsPopulated(String outputFolder, int timeout) throws IOException, InterruptedException + { + boolean result; + long now = System.currentTimeMillis(); + Path outDir = new Path("file://" + new File(outputFolder).getAbsolutePath()); + try (FileSystem fs = FileSystem.newInstance(outDir.toUri(), new Configuration())) { + List strings = Lists.newArrayList(); + while (System.currentTimeMillis() - now < timeout) { + if (fs.exists(outDir)) { + File file = new File(outputFolder); + if (file.list().length > 0) { + File file1 = new File(outputFolder + file.list()[0]); + strings = FileUtils.readLines(file1); + if (strings.size() != 0) { + break; + } + } + } + + Thread.sleep(500); + } + + result = fs.exists(outDir) && (strings.size() != 0); + } + + return result; + } + + + public static class Application implements StreamingApplication + { + String model; + + public Application(String model) + { + this.model = model; + } + + @Override + public void populateDAG(DAG dag, Configuration conf) + { + SQLExecEnvironment.getEnvironment() + .withModel(model) + .executeSQL(dag, "SELECT STREAM ROWTIME, PRODUCT FROM ORDERS"); + } + } + + public static class ApplicationSelectInsertWithAPI implements StreamingApplication + { + @Override + public void populateDAG(DAG dag, Configuration conf) + { + String schemaIn = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"id\",\"type\":\"Integer\"}," + + "{\"name\":\"Product\",\"type\":\"String\"}," + + "{\"name\":\"units\",\"type\":\"Integer\"}]}"; + String schemaOut = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime1\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"RowTime2\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"Product\",\"type\":\"String\"}]}"; + + SQLExecEnvironment.getEnvironment() + .registerTable("ORDERS", new FileEndpoint("src/test/resources/input.csv", + new CSVMessageFormat(schemaIn))) + .registerTable("SALES", new FileEndpoint(outputFolder, "out.tmp", new CSVMessageFormat(schemaOut))) + .registerFunction("APEXCONCAT", FileEndpointTest.class, "apex_concat_str") + .executeSQL(dag, "INSERT INTO SALES " + "SELECT STREAM ROWTIME, " + "FLOOR(ROWTIME TO DAY), " + + "APEXCONCAT('OILPAINT', SUBSTRING(PRODUCT, 6, 7)) " + "FROM ORDERS WHERE ID > 3 " + "AND " + + "PRODUCT LIKE 'paint%'"); + } + } + +} diff --git a/sql/src/test/java/org/apache/apex/malhar/sql/InputPOJO.java b/sql/src/test/java/org/apache/apex/malhar/sql/InputPOJO.java new file mode 100644 index 0000000000..7162e3128d --- /dev/null +++ b/sql/src/test/java/org/apache/apex/malhar/sql/InputPOJO.java @@ -0,0 +1,69 @@ +/** + * 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.apex.malhar.sql; + +import java.util.Date; + +public class InputPOJO +{ + private Date RowTime; + private int id; + private String Product; + private int units; + + public Date getRowTime() + { + return RowTime; + } + + public void setRowTime(Date rowTime) + { + RowTime = rowTime; + } + + public int getId() + { + return id; + } + + public void setId(int id) + { + this.id = id; + } + + public String getProduct() + { + return Product; + } + + public void setProduct(String product) + { + Product = product; + } + + public int getUnits() + { + return units; + } + + public void setUnits(int units) + { + this.units = units; + } +} diff --git a/sql/src/test/java/org/apache/apex/malhar/sql/KafkaEndpointTest.java b/sql/src/test/java/org/apache/apex/malhar/sql/KafkaEndpointTest.java new file mode 100644 index 0000000000..14eff7098b --- /dev/null +++ b/sql/src/test/java/org/apache/apex/malhar/sql/KafkaEndpointTest.java @@ -0,0 +1,362 @@ +/** + * 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.apex.malhar.sql; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Properties; +import java.util.TimeZone; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.apex.malhar.kafka.EmbeddedKafka; +import org.apache.apex.malhar.kafka.KafkaSinglePortInputOperator; +import org.apache.apex.malhar.kafka.KafkaSinglePortOutputOperator; +import org.apache.apex.malhar.sql.table.CSVMessageFormat; +import org.apache.apex.malhar.sql.table.KafkaEndpoint; +import org.apache.apex.malhar.sql.table.StreamEndpoint; + +import org.apache.hadoop.conf.Configuration; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; + +import com.datatorrent.api.DAG; +import com.datatorrent.api.LocalMode; +import com.datatorrent.api.StreamingApplication; +import com.datatorrent.contrib.formatter.CsvFormatter; +import com.datatorrent.contrib.parser.CsvParser; + +public class KafkaEndpointTest +{ + private final String testTopicData0 = "dataTopic0"; + private final String testTopicData1 = "dataTopic1"; + private final String testTopicResult = "resultTopic"; + + private EmbeddedKafka kafka; + + private TimeZone defaultTZ; + + @Before + public void setup() throws IOException + { + defaultTZ = TimeZone.getDefault(); + TimeZone.setDefault(TimeZone.getTimeZone("GMT")); + + kafka = new EmbeddedKafka(); + kafka.start(); + kafka.createTopic(testTopicData0); + kafka.createTopic(testTopicData1); + kafka.createTopic(testTopicResult); + } + + @After + public void tearDown() throws IOException + { + kafka.stop(); + + TimeZone.setDefault(defaultTZ); + } + + @Test + public void testApplicationSelectInsertWithAPI() throws Exception + { + try { + LocalMode lma = LocalMode.newInstance(); + Configuration conf = new Configuration(false); + lma.prepareDAG(new KafkaApplication(kafka.getBroker(), testTopicData0, testTopicResult), conf); + LocalMode.Controller lc = lma.getController(); + lc.runAsync(); + + kafka.publish(testTopicData0, Arrays.asList("15/02/2016 10:15:00 +0000,1,paint1,11", + "15/02/2016 10:16:00 +0000,2,paint2,12", + "15/02/2016 10:17:00 +0000,3,paint3,13", "15/02/2016 10:18:00 +0000,4,paint4,14", + "15/02/2016 10:19:00 +0000,5,paint5,15", "15/02/2016 10:10:00 +0000,6,abcde6,16")); + + // TODO: Workaround to add \r\n char to test results because of bug in CsvFormatter which adds new line char. + String[] expectedLines = new String[]{"15/02/2016 10:18:00 +0000,15/02/2016 00:00:00 +0000,OILPAINT4\r\n", + "15/02/2016 10:19:00 +0000,15/02/2016 00:00:00 +0000,OILPAINT5\r\n"}; + + List consume = kafka.consume(testTopicResult, 30000); + Assert.assertTrue(Arrays.deepEquals(consume.toArray(new String[consume.size()]), expectedLines)); + + lc.shutdown(); + } catch (Exception e) { + Assert.fail("constraint violations: " + e); + } + } + + @Test + public void testApplicationWithPortEndpoint() throws Exception + { + try { + LocalMode lma = LocalMode.newInstance(); + Configuration conf = new Configuration(false); + lma.prepareDAG(new KafkaPortApplication(kafka.getBroker(), testTopicData0, testTopicResult), conf); + LocalMode.Controller lc = lma.getController(); + lc.runAsync(); + + kafka.publish(testTopicData0, Arrays.asList("15/02/2016 10:15:00 +0000,1,paint1,11", + "15/02/2016 10:16:00 +0000,2,paint2,12", + "15/02/2016 10:17:00 +0000,3,paint3,13", "15/02/2016 10:18:00 +0000,4,paint4,14", + "15/02/2016 10:19:00 +0000,5,paint5,15", "15/02/2016 10:10:00 +0000,6,abcde6,16")); + + // TODO: Workaround to add \r\n char to test results because of bug in CsvFormatter which adds new line char. + String[] expectedLines = new String[]{"15/02/2016 10:18:00 +0000,15/02/2016 00:00:00 +0000,OILPAINT4\r\n", + "15/02/2016 10:19:00 +0000,15/02/2016 00:00:00 +0000,OILPAINT5\r\n"}; + + List consume = kafka.consume(testTopicResult, 30000); + + Assert.assertTrue(Arrays.deepEquals(consume.toArray(new String[consume.size()]), expectedLines)); + + lc.shutdown(); + } catch (Exception e) { + Assert.fail("constraint violations: " + e); + } + } + + @Test + public void testApplicationJoin() throws Exception + { + String sql = "INSERT INTO SALES " + + "SELECT STREAM A.ROWTIME, FLOOR(A.ROWTIME TO DAY), " + + "APEXCONCAT('OILPAINT', SUBSTRING(A.PRODUCT, 6, 7)), B.CATEGORY " + + "FROM ORDERS AS A " + + "JOIN CATEGORY AS B ON A.id = B.id " + + "WHERE A.id > 3 AND A.PRODUCT LIKE 'paint%'"; + try { + LocalMode lma = LocalMode.newInstance(); + Configuration conf = new Configuration(false); + lma.prepareDAG(new KafkaJoinApplication(kafka.getBroker(), testTopicData0, testTopicData1, + testTopicResult, sql), conf); + LocalMode.Controller lc = lma.getController(); + lc.runAsync(); + + kafka.publish(testTopicData0, Arrays.asList("15/02/2016 10:15:00 +0000,1,paint1,11", + "15/02/2016 10:16:00 +0000,2,paint2,12", + "15/02/2016 10:17:00 +0000,3,paint3,13", "15/02/2016 10:18:00 +0000,4,paint4,14", + "15/02/2016 10:19:00 +0000,5,paint5,15", "15/02/2016 10:10:00 +0000,6,abcde6,16")); + + kafka.publish(testTopicData1, Arrays.asList("1,ABC", + "2,DEF", + "3,GHI", "4,JKL", + "5,MNO", "6,PQR")); + + // TODO: Workaround to add \r\n char to test results because of bug in CsvFormatter which adds new line char. + String[] expectedLines = new String[]{"15/02/2016 10:18:00 +0000,15/02/2016 00:00:00 +0000,OILPAINT4,JKL\r\n", + "15/02/2016 10:19:00 +0000,15/02/2016 00:00:00 +0000,OILPAINT5,MNO\r\n"}; + + List consume = kafka.consume(testTopicResult, 30000); + + Assert.assertTrue(Arrays.deepEquals(consume.toArray(new String[consume.size()]), expectedLines)); + + lc.shutdown(); + } catch (Exception e) { + Assert.fail("constraint violations: " + e); + } + } + + @Test + public void testApplicationJoinFilter() throws Exception + { + String sql = "INSERT INTO SALES SELECT STREAM A.ROWTIME, FLOOR(A.ROWTIME TO DAY), " + + "APEXCONCAT('OILPAINT', SUBSTRING(A.PRODUCT, 6, 7)), B.CATEGORY " + + "FROM ORDERS AS A JOIN CATEGORY AS B ON A.id = B.id AND A.id > 3" + + "WHERE A.PRODUCT LIKE 'paint%'"; + + try { + LocalMode lma = LocalMode.newInstance(); + Configuration conf = new Configuration(false); + lma.prepareDAG(new KafkaJoinApplication(kafka.getBroker(), testTopicData0, testTopicData1, + testTopicResult, sql), conf); + LocalMode.Controller lc = lma.getController(); + lc.runAsync(); + + kafka.publish(testTopicData0, Arrays.asList("15/02/2016 10:15:00 +0000,1,paint1,11", + "15/02/2016 10:16:00 +0000,2,paint2,12", + "15/02/2016 10:17:00 +0000,3,paint3,13", "15/02/2016 10:18:00 +0000,4,paint4,14", + "15/02/2016 10:19:00 +0000,5,paint5,15", "15/02/2016 10:10:00 +0000,6,abcde6,16")); + + kafka.publish(testTopicData1, Arrays.asList("1,ABC", + "2,DEF", + "3,GHI", "4,JKL", + "5,MNO", "6,PQR")); + + // TODO: Workaround to add \r\n char to test results because of bug in CsvFormatter which adds new line char. + String[] expectedLines = new String[]{"15/02/2016 10:18:00 +0000,15/02/2016 00:00:00 +0000,OILPAINT4,JKL\r\n", + "15/02/2016 10:19:00 +0000,15/02/2016 00:00:00 +0000,OILPAINT5,MNO\r\n"}; + + List consume = kafka.consume(testTopicResult, 30000); + + Assert.assertTrue(Arrays.deepEquals(consume.toArray(new String[consume.size()]), expectedLines)); + + lc.shutdown(); + } catch (Exception e) { + Assert.fail("constraint violations: " + e); + } + } + + public static class KafkaApplication implements StreamingApplication + { + private String broker; + private String sourceTopic; + private String destTopic; + + public KafkaApplication(String broker, String sourceTopic, String destTopic) + { + this.broker = broker; + this.sourceTopic = sourceTopic; + this.destTopic = destTopic; + } + + @Override + public void populateDAG(DAG dag, Configuration conf) + { + String schemaIn = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"id\",\"type\":\"Integer\"}," + + "{\"name\":\"Product\",\"type\":\"String\"}," + + "{\"name\":\"units\",\"type\":\"Integer\"}]}"; + String schemaOut = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime1\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"RowTime2\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"Product\",\"type\":\"String\"}]}"; + + SQLExecEnvironment.getEnvironment() + .registerTable("ORDERS", new KafkaEndpoint(broker, sourceTopic, new CSVMessageFormat(schemaIn))) + .registerTable("SALES", new KafkaEndpoint(broker, destTopic, new CSVMessageFormat(schemaOut))) + .registerFunction("APEXCONCAT", FileEndpointTest.class, "apex_concat_str") + .executeSQL(dag, "INSERT INTO SALES " + "SELECT STREAM ROWTIME, " + "FLOOR(ROWTIME TO DAY), " + + "APEXCONCAT('OILPAINT', SUBSTRING(PRODUCT, 6, 7)) " + "FROM ORDERS WHERE ID > 3 " + "AND " + + "PRODUCT LIKE 'paint%'"); + } + } + + public static class KafkaJoinApplication implements StreamingApplication + { + private String broker; + private String sourceTopic0; + private String sourceTopic1; + private String destTopic; + private String sql; + + public KafkaJoinApplication(String broker, String sourceTopic0, String sourceTopic1, String destTopic, String sql) + { + this.broker = broker; + this.sourceTopic0 = sourceTopic0; + this.sourceTopic1 = sourceTopic1; + this.destTopic = destTopic; + this.sql = sql; + } + + @Override + public void populateDAG(DAG dag, Configuration conf) + { + String schemaIn0 = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"id\",\"type\":\"Integer\"}," + + "{\"name\":\"Product\",\"type\":\"String\"}," + + "{\"name\":\"units\",\"type\":\"Integer\"}]}"; + String schemaIn1 = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"id\",\"type\":\"Integer\"}," + + "{\"name\":\"Category\",\"type\":\"String\"}]}"; + String schemaOut = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime1\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"RowTime2\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"Product\",\"type\":\"String\"}," + + "{\"name\":\"Category\",\"type\":\"String\"}]}"; + + SQLExecEnvironment.getEnvironment() + .registerTable("ORDERS", new KafkaEndpoint(broker, sourceTopic0, new CSVMessageFormat(schemaIn0))) + .registerTable("CATEGORY", new KafkaEndpoint(broker, sourceTopic1, new CSVMessageFormat(schemaIn1))) + .registerTable("SALES", new KafkaEndpoint(broker, destTopic, new CSVMessageFormat(schemaOut))) + .registerFunction("APEXCONCAT", FileEndpointTest.class, "apex_concat_str") + .executeSQL(dag, sql); + } + } + + public static class KafkaPortApplication implements StreamingApplication + { + private String broker; + private String sourceTopic; + private String destTopic; + + public KafkaPortApplication(String broker, String sourceTopic, String destTopic) + { + this.broker = broker; + this.sourceTopic = sourceTopic; + this.destTopic = destTopic; + } + + @Override + public void populateDAG(DAG dag, Configuration conf) + { + String schemaIn = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"id\",\"type\":\"Integer\"}," + + "{\"name\":\"Product\",\"type\":\"String\"}," + + "{\"name\":\"units\",\"type\":\"Integer\"}]}"; + String schemaOut = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime1\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"RowTime2\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"Product\",\"type\":\"String\"}]}"; + + KafkaSinglePortInputOperator kafkaInput = dag.addOperator("KafkaInput", KafkaSinglePortInputOperator.class); + kafkaInput.setTopics(sourceTopic); + kafkaInput.setInitialOffset("EARLIEST"); + Properties props = new Properties(); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, broker); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, KafkaEndpoint.KEY_DESERIALIZER); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, KafkaEndpoint.VALUE_DESERIALIZER); + kafkaInput.setConsumerProps(props); + kafkaInput.setClusters(broker); + + CsvParser csvParser = dag.addOperator("CSVParser", CsvParser.class); + csvParser.setSchema(schemaIn); + + dag.addStream("KafkaToCSV", kafkaInput.outputPort, csvParser.in); + + CsvFormatter formatter = dag.addOperator("CSVFormatter", CsvFormatter.class); + formatter.setSchema(schemaOut); + + KafkaSinglePortOutputOperator kafkaOutput = dag.addOperator("KafkaOutput", KafkaSinglePortOutputOperator.class); + kafkaOutput.setTopic(destTopic); + + props = new Properties(); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, KafkaEndpoint.VALUE_SERIALIZER); + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, KafkaEndpoint.KEY_SERIALIZER); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, broker); + kafkaOutput.setProperties(props); + + dag.addStream("CSVToKafka", formatter.out, kafkaOutput.inputPort); + + SQLExecEnvironment.getEnvironment() + .registerTable("ORDERS", new StreamEndpoint(csvParser.out, InputPOJO.class)) + .registerTable("SALES", new StreamEndpoint(formatter.in, OutputPOJO.class)) + .registerFunction("APEXCONCAT", FileEndpointTest.class, "apex_concat_str") + .executeSQL(dag, "INSERT INTO SALES " + "SELECT STREAM ROWTIME, " + "FLOOR(ROWTIME TO DAY), " + + "APEXCONCAT('OILPAINT', SUBSTRING(PRODUCT, 6, 7)) " + "FROM ORDERS WHERE ID > 3 " + "AND " + + "PRODUCT LIKE 'paint%'"); + } + } +} diff --git a/sql/src/test/java/org/apache/apex/malhar/sql/OutputPOJO.java b/sql/src/test/java/org/apache/apex/malhar/sql/OutputPOJO.java new file mode 100644 index 0000000000..fdf78d78e9 --- /dev/null +++ b/sql/src/test/java/org/apache/apex/malhar/sql/OutputPOJO.java @@ -0,0 +1,59 @@ +/** + * 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.apex.malhar.sql; + + +import java.util.Date; + +public class OutputPOJO +{ + private Date RowTime1; + private Date RowTime2; + private String Product; + + public Date getRowTime1() + { + return RowTime1; + } + + public void setRowTime1(Date rowTime1) + { + RowTime1 = rowTime1; + } + + public Date getRowTime2() + { + return RowTime2; + } + + public void setRowTime2(Date rowTime2) + { + RowTime2 = rowTime2; + } + + public String getProduct() + { + return Product; + } + + public void setProduct(String product) + { + Product = product; + } +} diff --git a/sql/src/test/java/org/apache/apex/malhar/sql/SerDeTest.java b/sql/src/test/java/org/apache/apex/malhar/sql/SerDeTest.java new file mode 100644 index 0000000000..62d2a4d21e --- /dev/null +++ b/sql/src/test/java/org/apache/apex/malhar/sql/SerDeTest.java @@ -0,0 +1,223 @@ +/** + * 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.apex.malhar.sql; + +import java.io.File; +import java.io.IOException; +import java.util.Properties; + +import org.junit.Test; + +import org.apache.apex.malhar.kafka.KafkaSinglePortInputOperator; +import org.apache.apex.malhar.kafka.KafkaSinglePortOutputOperator; +import org.apache.apex.malhar.sql.table.CSVMessageFormat; +import org.apache.apex.malhar.sql.table.Endpoint; +import org.apache.apex.malhar.sql.table.FileEndpoint; +import org.apache.apex.malhar.sql.table.KafkaEndpoint; +import org.apache.apex.malhar.sql.table.StreamEndpoint; + +import org.apache.commons.io.FileUtils; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; + +import com.datatorrent.contrib.formatter.CsvFormatter; +import com.datatorrent.contrib.parser.CsvParser; +import com.datatorrent.stram.plan.logical.LogicalPlan; + +public class SerDeTest +{ + @Test + public void testSQLWithApexFactory() throws IOException, ClassNotFoundException + { + File modelFile = new File("src/test/resources/model/model_file_csv.json"); + String model = FileUtils.readFileToString(modelFile); + + LogicalPlan dag = new LogicalPlan(); + SQLExecEnvironment.getEnvironment() + .withModel(model) + .executeSQL(dag, "SELECT STREAM ROWTIME, PRODUCT FROM ORDERS"); + + dag.validate(); + } + + @Test + public void testSQLWithAPI() throws ClassNotFoundException, IOException + { + LogicalPlan dag = new LogicalPlan(); + + String schema = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy hh:mm:ss Z\"}}," + + "{\"name\":\"id\",\"type\":\"Integer\"},{\"name\":\"Product\",\"type\":\"String\"}," + + "{\"name\":\"units\",\"type\":\"Integer\"}]}"; + + Endpoint endpoint = new FileEndpoint("dummyFilePath", new CSVMessageFormat(schema)); + + SQLExecEnvironment.getEnvironment() + .registerTable("ORDERS", endpoint) + .executeSQL(dag, "SELECT STREAM FLOOR(ROWTIME TO HOUR), SUBSTRING(PRODUCT, 0, 5) FROM ORDERS WHERE id > 3"); + + dag.validate(); + } + + @Test + public void testSQLSelectInsertWithAPI() throws IOException, ClassNotFoundException + { + LogicalPlan dag = new LogicalPlan(); + + String schemaIn = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy hh:mm:ss\"}}," + + "{\"name\":\"id\",\"type\":\"Integer\"}," + + "{\"name\":\"Product\",\"type\":\"String\"}," + + "{\"name\":\"units\",\"type\":\"Integer\"}]}"; + String schemaOut = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy hh:mm:ss\"}}," + + "{\"name\":\"Product\",\"type\":\"String\"}]}"; + + SQLExecEnvironment.getEnvironment() + .registerTable("ORDERS", new FileEndpoint("dummyFilePathInput", new CSVMessageFormat(schemaIn))) + .registerTable("SALES", new FileEndpoint("dummyFilePathOutput", "out.tmp", new CSVMessageFormat(schemaOut))) + .executeSQL(dag, "INSERT INTO SALES SELECT STREAM FLOOR(ROWTIME TO HOUR), SUBSTRING(PRODUCT, 0, 5) " + + "FROM ORDERS WHERE id > 3"); + + dag.validate(); + } + + @Test + public void testJoin() throws IOException, ClassNotFoundException + { + LogicalPlan dag = new LogicalPlan(); + String schemaIn0 = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy hh:mm:ss Z\"}}," + + "{\"name\":\"id\",\"type\":\"Integer\"}," + + "{\"name\":\"Product\",\"type\":\"String\"}," + + "{\"name\":\"units\",\"type\":\"Integer\"}]}"; + String schemaIn1 = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"id\",\"type\":\"Integer\"}," + + "{\"name\":\"Category\",\"type\":\"String\"}]}"; + String schemaOut = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime1\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy hh:mm:ss Z\"}}," + + "{\"name\":\"RowTime2\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy hh:mm:ss Z\"}}," + + "{\"name\":\"Product\",\"type\":\"String\"}," + + "{\"name\":\"Category\",\"type\":\"String\"}]}"; + + String sql = "INSERT INTO SALES SELECT STREAM A.ROWTIME, FLOOR(A.ROWTIME TO DAY), " + + "APEXCONCAT('OILPAINT', SUBSTRING(A.PRODUCT, 6, 7)), B.CATEGORY " + + "FROM ORDERS AS A " + + "JOIN CATEGORY AS B ON A.id = B.id " + + "WHERE A.id > 3 AND A.PRODUCT LIKE 'paint%'"; + + SQLExecEnvironment.getEnvironment() + .registerTable("ORDERS", new KafkaEndpoint("localhost:9092", "testdata0", new CSVMessageFormat(schemaIn0))) + .registerTable("CATEGORY", new KafkaEndpoint("localhost:9092", "testdata1", new CSVMessageFormat(schemaIn1))) + .registerTable("SALES", new KafkaEndpoint("localhost:9092", "testresult", new CSVMessageFormat(schemaOut))) + .registerFunction("APEXCONCAT", FileEndpointTest.class, "apex_concat_str") + .executeSQL(dag, sql); + + dag.validate(); + } + + @Test + public void testJoinFilter() throws IOException, ClassNotFoundException + { + LogicalPlan dag = new LogicalPlan(); + String schemaIn0 = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy hh:mm:ss Z\"}}," + + "{\"name\":\"id\",\"type\":\"Integer\"}," + + "{\"name\":\"Product\",\"type\":\"String\"}," + + "{\"name\":\"units\",\"type\":\"Integer\"}]}"; + String schemaIn1 = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"id\",\"type\":\"Integer\"}," + + "{\"name\":\"Category\",\"type\":\"String\"}]}"; + String schemaOut = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime1\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy hh:mm:ss Z\"}}," + + "{\"name\":\"RowTime2\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy hh:mm:ss Z\"}}," + + "{\"name\":\"Product\",\"type\":\"String\"}," + + "{\"name\":\"Category\",\"type\":\"String\"}]}"; + + String sql = "INSERT INTO SALES SELECT STREAM A.ROWTIME, FLOOR(A.ROWTIME TO DAY), " + + "APEXCONCAT('OILPAINT', SUBSTRING(A.PRODUCT, 6, 7)), B.CATEGORY " + + "FROM ORDERS AS A JOIN CATEGORY AS B ON A.id = B.id AND A.id > 3" + + "WHERE A.PRODUCT LIKE 'paint%'"; + + SQLExecEnvironment.getEnvironment() + .registerTable("ORDERS", new KafkaEndpoint("localhost:9092", "testdata0", new CSVMessageFormat(schemaIn0))) + .registerTable("CATEGORY", new KafkaEndpoint("localhost:9092", "testdata1", new CSVMessageFormat(schemaIn1))) + .registerTable("SALES", new KafkaEndpoint("localhost:9092", "testresult", new CSVMessageFormat(schemaOut))) + .registerFunction("APEXCONCAT", FileEndpointTest.class, "apex_concat_str") + .executeSQL(dag, sql); + + dag.validate(); + } + + @Test + public void testPortEndpoint() throws IOException, ClassNotFoundException + { + LogicalPlan dag = new LogicalPlan(); + + String schemaIn = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy hh:mm:ss Z\"}}," + + "{\"name\":\"id\",\"type\":\"Integer\"}," + + "{\"name\":\"Product\",\"type\":\"String\"}," + + "{\"name\":\"units\",\"type\":\"Integer\"}]}"; + String schemaOut = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime1\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy hh:mm:ss Z\"}}," + + "{\"name\":\"RowTime2\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy hh:mm:ss Z\"}}," + + "{\"name\":\"Product\",\"type\":\"String\"}]}"; + + KafkaSinglePortInputOperator kafkaInput = dag.addOperator("KafkaInput", KafkaSinglePortInputOperator.class); + kafkaInput.setTopics("testdata0"); + kafkaInput.setInitialOffset("EARLIEST"); + Properties props = new Properties(); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, KafkaEndpoint.KEY_DESERIALIZER); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, KafkaEndpoint.VALUE_DESERIALIZER); + kafkaInput.setConsumerProps(props); + kafkaInput.setClusters("localhost:9092"); + + CsvParser csvParser = dag.addOperator("CSVParser", CsvParser.class); + csvParser.setSchema(schemaIn); + + dag.addStream("KafkaToCSV", kafkaInput.outputPort, csvParser.in); + + CsvFormatter formatter = dag.addOperator("CSVFormatter", CsvFormatter.class); + formatter.setSchema(schemaOut); + + KafkaSinglePortOutputOperator kafkaOutput = dag.addOperator("KafkaOutput", KafkaSinglePortOutputOperator.class); + kafkaOutput.setTopic("testresult"); + + props = new Properties(); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, KafkaEndpoint.VALUE_SERIALIZER); + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, KafkaEndpoint.KEY_SERIALIZER); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + kafkaOutput.setProperties(props); + + dag.addStream("CSVToKafka", formatter.out, kafkaOutput.inputPort); + + SQLExecEnvironment.getEnvironment() + .registerTable("ORDERS", new StreamEndpoint(csvParser.out, InputPOJO.class)) + .registerTable("SALES", new StreamEndpoint(formatter.in, OutputPOJO.class)) + .registerFunction("APEXCONCAT", FileEndpointTest.class, "apex_concat_str") + .executeSQL(dag, "INSERT INTO SALES " + "SELECT STREAM ROWTIME, " + "FLOOR(ROWTIME TO DAY), " + + "APEXCONCAT('OILPAINT', SUBSTRING(PRODUCT, 6, 7)) " + "FROM ORDERS WHERE ID > 3 " + "AND " + + "PRODUCT LIKE 'paint%'"); + + dag.validate(); + } +} diff --git a/sql/src/test/java/org/apache/apex/malhar/sql/StreamEndpointTest.java b/sql/src/test/java/org/apache/apex/malhar/sql/StreamEndpointTest.java new file mode 100644 index 0000000000..1e9a1f8f13 --- /dev/null +++ b/sql/src/test/java/org/apache/apex/malhar/sql/StreamEndpointTest.java @@ -0,0 +1,179 @@ +/** + * 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.apex.malhar.sql; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Date; +import java.util.List; +import java.util.Properties; +import java.util.TimeZone; + +import javax.validation.ConstraintViolationException; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.apex.malhar.kafka.EmbeddedKafka; +import org.apache.apex.malhar.kafka.KafkaSinglePortInputOperator; +import org.apache.apex.malhar.kafka.KafkaSinglePortOutputOperator; +import org.apache.apex.malhar.sql.table.KafkaEndpoint; +import org.apache.apex.malhar.sql.table.StreamEndpoint; + +import org.apache.hadoop.conf.Configuration; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; + +import com.google.common.collect.ImmutableMap; + +import com.datatorrent.api.DAG; +import com.datatorrent.api.LocalMode; +import com.datatorrent.api.StreamingApplication; +import com.datatorrent.contrib.formatter.CsvFormatter; +import com.datatorrent.contrib.parser.CsvParser; + +public class StreamEndpointTest +{ + private String testTopicData0 = "dataTopic0"; + private String testTopicResult = "resultTopic"; + + private EmbeddedKafka kafka; + + private TimeZone defaultTZ; + + @Before + public void setup() throws IOException + { + defaultTZ = TimeZone.getDefault(); + TimeZone.setDefault(TimeZone.getTimeZone("GMT")); + + kafka = new EmbeddedKafka(); + kafka.start(); + kafka.createTopic(testTopicData0); + kafka.createTopic(testTopicResult); + } + + @After + public void tearDown() throws IOException + { + kafka.stop(); + + TimeZone.setDefault(defaultTZ); + } + + @Test + public void testApplicationWithPortEndpoint() + { + try { + LocalMode lma = LocalMode.newInstance(); + Configuration conf = new Configuration(false); + lma.prepareDAG(new KafkaPortApplication(kafka.getBroker(), testTopicData0, testTopicResult), conf); + LocalMode.Controller lc = lma.getController(); + lc.runAsync(); + + kafka.publish(testTopicData0, Arrays.asList("15/02/2016 10:15:00 +0000,1,paint1,11", + "15/02/2016 10:16:00 +0000,2,paint2,12", + "15/02/2016 10:17:00 +0000,3,paint3,13", "15/02/2016 10:18:00 +0000,4,paint4,14", + "15/02/2016 10:19:00 +0000,5,paint5,15", "15/02/2016 10:10:00 +0000,6,abcde6,16")); + + // TODO: Workaround to add \r\n char to test results because of bug in CsvFormatter which adds new line char. + String[] expectedLines = new String[] {"15/02/2016 10:18:00 +0000,15/02/2016 00:00:00 +0000,OILPAINT4\r\n", + "15/02/2016 10:19:00 +0000,15/02/2016 00:00:00 +0000,OILPAINT5\r\n"}; + + List consume = kafka.consume(testTopicResult, 30000); + + Assert.assertTrue(Arrays.deepEquals(consume.toArray(new String[consume.size()]), expectedLines)); + + lc.shutdown(); + } catch (ConstraintViolationException e) { + Assert.fail("constraint violations: " + e.getConstraintViolations()); + } catch (Exception e) { + Assert.fail("Exception: " + e); + } + } + + public static class KafkaPortApplication implements StreamingApplication + { + private String broker; + private String sourceTopic; + private String destTopic; + + public KafkaPortApplication(String broker, String sourceTopic, String destTopic) + { + this.broker = broker; + this.sourceTopic = sourceTopic; + this.destTopic = destTopic; + } + + @Override + public void populateDAG(DAG dag, Configuration conf) + { + String schemaIn = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"id\",\"type\":\"Integer\"}," + + "{\"name\":\"Product\",\"type\":\"String\"}," + + "{\"name\":\"units\",\"type\":\"Integer\"}]}"; + String schemaOut = "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[" + + "{\"name\":\"RowTime1\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"RowTime2\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy HH:mm:ss Z\"}}," + + "{\"name\":\"Product\",\"type\":\"String\"}]}"; + + KafkaSinglePortInputOperator kafkaInput = dag.addOperator("KafkaInput", KafkaSinglePortInputOperator.class); + kafkaInput.setTopics(sourceTopic); + kafkaInput.setInitialOffset("EARLIEST"); + Properties props = new Properties(); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, broker); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, KafkaEndpoint.KEY_DESERIALIZER); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, KafkaEndpoint.VALUE_DESERIALIZER); + kafkaInput.setConsumerProps(props); + kafkaInput.setClusters(broker); + + CsvParser csvParser = dag.addOperator("CSVParser", CsvParser.class); + csvParser.setSchema(schemaIn); + + dag.addStream("KafkaToCSV", kafkaInput.outputPort, csvParser.in); + + CsvFormatter formatter = dag.addOperator("CSVFormatter", CsvFormatter.class); + formatter.setSchema(schemaOut); + + KafkaSinglePortOutputOperator kafkaOutput = dag.addOperator("KafkaOutput", KafkaSinglePortOutputOperator.class); + kafkaOutput.setTopic(destTopic); + + props = new Properties(); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, KafkaEndpoint.VALUE_SERIALIZER); + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, KafkaEndpoint.KEY_SERIALIZER); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, broker); + kafkaOutput.setProperties(props); + + dag.addStream("CSVToKafka", formatter.out, kafkaOutput.inputPort); + + SQLExecEnvironment.getEnvironment() + .registerTable("ORDERS", new StreamEndpoint(csvParser.out, InputPOJO.class)) + .registerTable("SALES", new StreamEndpoint(formatter.in, + ImmutableMap.of("RowTime1", Date.class, "RowTime2", Date.class, "Product", String.class))) + .registerFunction("APEXCONCAT", FileEndpointTest.class, "apex_concat_str") + .executeSQL(dag, "INSERT INTO SALES " + "SELECT STREAM ROWTIME, " + "FLOOR(ROWTIME TO DAY), " + + "APEXCONCAT('OILPAINT', SUBSTRING(PRODUCT, 6, 7)) " + "FROM ORDERS WHERE ID > 3 " + "AND " + + "PRODUCT LIKE 'paint%'"); + } + } +} diff --git a/sql/src/test/java/org/apache/apex/malhar/sql/codegen/BeanClassGeneratorTest.java b/sql/src/test/java/org/apache/apex/malhar/sql/codegen/BeanClassGeneratorTest.java index 8fcb7f8692..b5cd378a65 100644 --- a/sql/src/test/java/org/apache/apex/malhar/sql/codegen/BeanClassGeneratorTest.java +++ b/sql/src/test/java/org/apache/apex/malhar/sql/codegen/BeanClassGeneratorTest.java @@ -1,6 +1,20 @@ /** - * Copyright (c) 2015 DataTorrent, Inc. - * All rights reserved. + * 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.apex.malhar.sql.codegen; @@ -17,6 +31,7 @@ import org.junit.rules.TestWatcher; import org.junit.runner.Description; +import org.apache.apex.malhar.lib.utils.ClassLoaderUtils; import org.apache.apex.malhar.sql.schema.TupleSchemaRegistry; import static org.junit.Assert.assertEquals; @@ -60,7 +75,7 @@ public void test() throws IOException, JSONException, IllegalAccessException, In byte[] beanClass = BeanClassGenerator.createAndWriteBeanClass(addressClassName, schema.fieldList); - Class clazz = BeanClassGenerator.readBeanClass(addressClassName, beanClass); + Class clazz = ClassLoaderUtils.readBeanClass(addressClassName, beanClass); Object o = clazz.newInstance(); Field f = clazz.getDeclaredField("streetNumber"); @@ -91,7 +106,7 @@ public void testPrimitive() throws IOException, JSONException, IllegalAccessExce byte[] beanClass = BeanClassGenerator.createAndWriteBeanClass(addressClassName, schema.fieldList); - Class clazz = BeanClassGenerator.readBeanClass(addressClassName, beanClass); + Class clazz = ClassLoaderUtils.readBeanClass(addressClassName, beanClass); Object o = clazz.newInstance(); Field f = clazz.getDeclaredField("streetNumber"); diff --git a/sql/src/test/resources/input.csv b/sql/src/test/resources/input.csv new file mode 100644 index 0000000000..c4786d1a02 --- /dev/null +++ b/sql/src/test/resources/input.csv @@ -0,0 +1,6 @@ +15/02/2016 10:15:00 +0000,1,paint1,11 +15/02/2016 10:16:00 +0000,2,paint2,12 +15/02/2016 10:17:00 +0000,3,paint3,13 +15/02/2016 10:18:00 +0000,4,paint4,14 +15/02/2016 10:19:00 +0000,5,paint5,15 +15/02/2016 10:10:00 +0000,6,abcde6,16 diff --git a/sql/src/test/resources/log4j.properties b/sql/src/test/resources/log4j.properties new file mode 100644 index 0000000000..8ea3cfeb3d --- /dev/null +++ b/sql/src/test/resources/log4j.properties @@ -0,0 +1,50 @@ +# +# 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. +# + +log4j.rootLogger=DEBUG,CONSOLE + +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} %M - %m%n +log4j.appender.CONSOLE.threshold=WARN +test.log.console.threshold=WARN + +log4j.appender.RFA=org.apache.log4j.RollingFileAppender +log4j.appender.RFA.layout=org.apache.log4j.PatternLayout +log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} %M - %m%n +log4j.appender.RFA.File=/tmp/app.log + +# to enable, add SYSLOG to rootLogger +log4j.appender.SYSLOG=org.apache.log4j.net.SyslogAppender +log4j.appender.SYSLOG.syslogHost=127.0.0.1 +log4j.appender.SYSLOG.layout=org.apache.log4j.PatternLayout +log4j.appender.SYSLOG.layout.conversionPattern=${dt.cid} %-5p [%t] %c{2} %x - %m%n +log4j.appender.SYSLOG.Facility=LOCAL1 + +log4j.logger.org=info +#log4j.logger.org.apache.commons.beanutils=warn +log4j.logger.com.datatorrent=INFO +log4j.logger.org.apache.apex=INFO + +log4j.logger.org.apache.calcite=WARN +log4j.logger.org.apache.kafka=WARN +log4j.logger.org.I0Itec.zkclient.ZkClient=WARN +log4j.logger.org.apache.zookeeper=WARN +log4j.logger.kafka=WARN +log4j.logger.kafka.consumer=WARN diff --git a/sql/src/test/resources/model/model_file_csv.json b/sql/src/test/resources/model/model_file_csv.json new file mode 100644 index 0000000000..beba18dbe1 --- /dev/null +++ b/sql/src/test/resources/model/model_file_csv.json @@ -0,0 +1,27 @@ +{ + "version": "1.0", + "defaultSchema": "APEX", + "schemas": [{ + "name": "APEX", + "tables": [ + { + "name": "ORDERS", + "type": "custom", + "factory": "org.apache.apex.malhar.sql.schema.ApexSQLTableFactory", + "stream": { + "stream": true + }, + "operand": { + "endpoint": "file", + "messageFormat": "csv", + "endpointOperands": { + "directory": "src/test/resources/input.csv" + }, + "messageFormatOperands": { + "schema": "{\"separator\":\",\",\"quoteChar\":\"\\\"\",\"fields\":[{\"name\":\"RowTime\",\"type\":\"Date\",\"constraints\":{\"format\":\"dd/MM/yyyy hh:mm:ss\"}},{\"name\":\"id\",\"type\":\"Integer\"},{\"name\":\"Product\",\"type\":\"String\"},{\"name\":\"units\",\"type\":\"Integer\"}]}" + } + } + } + ] + }] +}