You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@apex.apache.org by th...@apache.org on 2016/03/25 14:17:03 UTC

[1/3] incubator-apex-malhar git commit: APEXMALHAR-2010 Add Tranform operator

Repository: incubator-apex-malhar
Updated Branches:
  refs/heads/devel-3 a8fbcac62 -> d9abee962


APEXMALHAR-2010 Add Tranform operator

Added expression interface and java expression parser.
Added expression support in PojoUtils.
Added unit tests for Expression support in PojoUtils.
Added transform operator.
Added unit tests for transform operator.


Project: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/commit/47df5ebc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/tree/47df5ebc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/diff/47df5ebc

Branch: refs/heads/devel-3
Commit: 47df5ebc676b8f3b49c34cf80bf5c064fb7dad6a
Parents: d3a7063
Author: chinmaykolhatkar <ch...@datatorrent.com>
Authored: Tue Mar 8 17:03:54 2016 +0530
Committer: chinmaykolhatkar <ch...@datatorrent.com>
Committed: Mon Mar 21 15:18:59 2016 +0530

----------------------------------------------------------------------
 .../datatorrent/lib/expression/Expression.java  |  61 ++++
 .../lib/expression/JavaExpressionParser.java    | 159 +++++++++
 .../lib/transform/TransformOperator.java        | 296 +++++++++++++++
 .../com/datatorrent/lib/util/PojoUtils.java     | 153 +++++---
 .../lib/transform/TransformOperatorAppTest.java | 130 +++++++
 .../lib/transform/TransformOperatorTest.java    | 356 +++++++++++++++++++
 .../com/datatorrent/lib/util/PojoUtilsTest.java | 103 +++++-
 7 files changed, 1198 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/47df5ebc/library/src/main/java/com/datatorrent/lib/expression/Expression.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/com/datatorrent/lib/expression/Expression.java b/library/src/main/java/com/datatorrent/lib/expression/Expression.java
new file mode 100644
index 0000000..cc29e15
--- /dev/null
+++ b/library/src/main/java/com/datatorrent/lib/expression/Expression.java
@@ -0,0 +1,61 @@
+/**
+ * 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 com.datatorrent.lib.expression;
+
+/**
+ * This is the Expression interface, object of which will be returned after evaluating expression.
+ * The interface can be used to execute given expression.
+ *
+ * @param <O> This is generic class return type of execute method of expression.
+ */
+public interface Expression<O>
+{
+  /**
+   * Method which contains the compiled and executable code corresponding to provided expression.
+   *
+   * @param obj Object on which expression will be executed.
+   * @return Returns result of expression of type O.
+   */
+  O execute(Object obj);
+
+  /**
+   * Defines an interface for how an expression should be parsed.
+   */
+  interface ExpressionParser
+  {
+    /**
+     * The method should convert given expression to a code which can be compiled for execution.
+     *
+     * @param expression Original expression which needs to be converted to compilable code.
+     * @param objectType Type of the object which will be used in expression.
+     * @param returnType Return type of given expression.
+     * @return Returns a java compilable code of given expression.
+     */
+    String convertToCompilableExpression(String expression, Class<?> objectType, Class<?> returnType);
+
+    /**
+     * This method provides way in which the object in action is placed in expression.
+     * The method also defined how should the object placeholder get converted in compilable code.
+     *
+     * @param exprObjPlaceholder String placeholder that can be used in expression to reference the object.
+     * @param codeObjPlaceholder String placeholder that will be used in code to replace exprObjectPlaceholder.
+     */
+    void setInputObjectPlaceholder(String exprObjPlaceholder, String codeObjPlaceholder);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/47df5ebc/library/src/main/java/com/datatorrent/lib/expression/JavaExpressionParser.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/com/datatorrent/lib/expression/JavaExpressionParser.java b/library/src/main/java/com/datatorrent/lib/expression/JavaExpressionParser.java
new file mode 100644
index 0000000..c11a0e9
--- /dev/null
+++ b/library/src/main/java/com/datatorrent/lib/expression/JavaExpressionParser.java
@@ -0,0 +1,159 @@
+/**
+ * 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 com.datatorrent.lib.expression;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.datatorrent.lib.util.KeyValPair;
+
+/**
+ * Defines how quasi-Java Expression should be parsed.
+ */
+public class JavaExpressionParser implements Expression.ExpressionParser
+{
+  private static final Logger logger = LoggerFactory.getLogger(JavaExpressionParser.class);
+
+  private static final String GET = "get";
+  private static final String IS = "is";
+
+  private final String variablePlaceholderPattern = "\\{(.*?)\\}";
+
+  private String exprObjPlaceholder;
+  private String codeObjPlaceholder;
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public String convertToCompilableExpression(String expression, Class<?> objectType, Class<?> returnType)
+  {
+    if (expression.startsWith(".")) {
+      expression = expression.substring(1);
+    }
+
+    if (expression.isEmpty()) {
+      throw new IllegalArgumentException("The getter expression: \"" + expression + "\" is invalid.");
+    }
+
+    Pattern entry = Pattern.compile(variablePlaceholderPattern);
+    Matcher matcher = entry.matcher(expression);
+    StringBuffer sb = new StringBuffer();
+
+    while (matcher.find()) {
+      if (matcher.groupCount() == 1) {
+        matcher.appendReplacement(sb, getObjectJavaExpression(matcher.group(1), objectType));
+      } else {
+        throw new RuntimeException("Invalid expression: " + matcher.group());
+      }
+    }
+
+    matcher.appendTail(sb);
+
+    if (sb.toString().equals(expression)) {
+      // This is a simple expression. No object placeholder. create proper expression.
+      if (!expression.startsWith(this.exprObjPlaceholder + ".")) {
+        expression = this.exprObjPlaceholder + "." + expression;
+      }
+      String tempExpr = getObjectJavaExpression(expression, objectType);
+      sb.setLength(0);
+      sb.append(tempExpr.replace("\\$", "$"));
+    }
+
+    return "return ((" + returnType.getName().replace("$", "\\$") + ")" + sb.toString() + ");";
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void setInputObjectPlaceholder(String exprObjPlaceholder, String codeObjPlaceholder)
+  {
+    this.exprObjPlaceholder = exprObjPlaceholder;
+    this.codeObjPlaceholder = codeObjPlaceholder;
+  }
+
+  private String getObjectJavaExpression(String exp, Class objectType)
+  {
+    StringBuilder sb = new StringBuilder();
+
+    String[] split = exp.split("\\.");
+    Class<?> currentClassType = objectType;
+
+    boolean first = true;
+    for (String field : split) {
+      if (first) {
+        first = false;
+      } else {
+        sb.append(".");
+      }
+      if (field.equals(exprObjPlaceholder)) {
+        // Replace with object type
+        sb.append("((").append(objectType.getName().replace("$", "\\$")).append(")").append("(")
+            .append(codeObjPlaceholder).append("))");
+        currentClassType = objectType;
+      } else {
+        KeyValPair<String, ? extends Class<?>> getter = getGetterForVariable(field, currentClassType);
+        sb.append(getter.getKey());
+        currentClassType = getter.getValue();
+      }
+    }
+
+    return sb.toString();
+  }
+
+  private KeyValPair<String, ? extends Class<?>> getGetterForVariable(String var, Class<?> inputClassType)
+  {
+    try {
+      final Field field = inputClassType.getField(var);
+      if (Modifier.isPublic(field.getModifiers())) {
+        return new KeyValPair<>(var, field.getType());
+      }
+      logger.debug("Field {} is not publicly accessible. Proceeding to locate a getter method.", var);
+    } catch (NoSuchFieldException ex) {
+      logger.debug("{} does not have field {}. Proceeding to locate a getter method.", inputClassType.getName(), var);
+    }
+
+    String[] methodAccessors = new String[]{GET, IS};
+
+    for (String m : methodAccessors) {
+      String methodName = m + var.substring(0, 1).toUpperCase() + var.substring(1);
+      try {
+        Method method = inputClassType.getMethod(methodName);
+        if (Modifier.isPublic(method.getModifiers())) {
+          return new KeyValPair<>(methodName + "()", method.getReturnType());
+        }
+        logger.debug("Method {} of {} is not accessible. Proceeding to locate another getter method.", methodName,
+            inputClassType.getName());
+      } catch (NoSuchMethodException ex) {
+        logger.debug("{} does not have method {}. Proceeding to locate another getter method", inputClassType.getName(),
+            methodName);
+      }
+    }
+
+    return new KeyValPair<>(var, inputClassType);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/47df5ebc/library/src/main/java/com/datatorrent/lib/transform/TransformOperator.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/com/datatorrent/lib/transform/TransformOperator.java b/library/src/main/java/com/datatorrent/lib/transform/TransformOperator.java
new file mode 100644
index 0000000..5b1a6c7
--- /dev/null
+++ b/library/src/main/java/com/datatorrent/lib/transform/TransformOperator.java
@@ -0,0 +1,296 @@
+/**
+ * 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 com.datatorrent.lib.transform;
+
+import java.lang.reflect.Field;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import javax.validation.constraints.NotNull;
+
+import org.apache.commons.lang3.ClassUtils;
+
+import com.datatorrent.api.Context;
+import com.datatorrent.api.DefaultInputPort;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.Operator;
+import com.datatorrent.api.annotation.InputPortFieldAnnotation;
+import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
+import com.datatorrent.common.util.BaseOperator;
+import com.datatorrent.lib.expression.Expression;
+import com.datatorrent.lib.util.PojoUtils;
+
+/**
+ * This operator can transform given POJO using provided expressions and
+ * return a final POJO as a return of transformation process.
+ *
+ * Following are the mandatory fields that needs to be set for TransformOperator to work:
+ * <ul>
+ *   <li><b>expressionMap</b> : Set how the transformation should happen</li>
+ *   <li><b>inputPort.attr.TUPLE_CLASS</b>: Set class type at input port</li>
+ *   <li><b>outputPort.attr.TUPLE_CLASS</b> : Set class type at output port</li>
+ * </ul>
+ *
+ * The operator uses interaction via {@link Expression} and {@link PojoUtils} to transform given POJO.
+ */
+public class TransformOperator extends BaseOperator implements Operator.ActivationListener
+{
+  @NotNull
+  private Map<String, String> expressionMap = new HashMap<>();
+  private List<String> expressionFuntions = new LinkedList<>();
+  private boolean copyMatchingFields = true;
+
+  private transient Map<PojoUtils.Setter, Expression> transformationMap = new HashMap<>();
+  private Class<?> inputClass;
+  private Class<?> outputClass;
+
+  public TransformOperator()
+  {
+    expressionFuntions.add("java.lang.Math.*");
+    expressionFuntions.add("org.apache.commons.lang3.StringUtils.*");
+    expressionFuntions.add("org.apache.commons.lang3.StringEscapeUtils.*");
+    expressionFuntions.add("org.apache.commons.lang3.time.DurationFormatUtils.*");
+    expressionFuntions.add("org.apache.commons.lang3.time.DateFormatUtils.*");
+  }
+
+  @InputPortFieldAnnotation(schemaRequired = true)
+  public final transient DefaultInputPort<Object> input = new DefaultInputPort<Object>()
+  {
+    @Override
+    public void setup(Context.PortContext context)
+    {
+      inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
+    }
+
+    @Override
+    public void process(Object o)
+    {
+      processTuple(o);
+    }
+  };
+
+  @OutputPortFieldAnnotation(schemaRequired = true)
+  public final transient DefaultOutputPort<Object> output = new DefaultOutputPort<Object>()
+  {
+    @Override public void setup(Context.PortContext context)
+    {
+      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
+    }
+  };
+
+  protected void processTuple(Object in)
+  {
+    if (!inputClass.isAssignableFrom(in.getClass())) {
+      throw new RuntimeException(
+          "Unexpected tuple received. Received class: " + in.getClass() + ". Expected class: " + inputClass.getClass());
+    }
+
+    Object out;
+    try {
+      out = outputClass.newInstance();
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new RuntimeException("Failed to create new object", e);
+    }
+
+    for (Map.Entry<PojoUtils.Setter, Expression> entry : transformationMap.entrySet()) {
+      PojoUtils.Setter set = entry.getKey();
+      Expression expr = entry.getValue();
+      set.set(out, expr.execute(in));
+    }
+
+    output.emit(out);
+  }
+
+  @Override
+  public void activate(Context context)
+  {
+    if (copyMatchingFields) {
+      Field[] declaredFields = outputClass.getDeclaredFields();
+      for (Field outputField : declaredFields) {
+        String outputFieldName = outputField.getName();
+        if (!expressionMap.containsKey(outputFieldName)) {
+          try {
+            Field inputField = inputClass.getDeclaredField(outputFieldName);
+            if (inputField.getType() == outputField.getType()) {
+              expressionMap.put(outputFieldName, inputField.getName());
+            }
+          } catch (NoSuchFieldException e) {
+            continue;
+          }
+        }
+      }
+    }
+
+    for (Map.Entry<String, String> entry : expressionMap.entrySet()) {
+      String field = entry.getKey();
+      String expr = entry.getValue();
+
+      // Generate output setter
+      Field f;
+      try {
+        f = outputClass.getDeclaredField(field);
+      } catch (NoSuchFieldException e) {
+        throw new RuntimeException("Failed to get output field info", e);
+      }
+
+      Class c = ClassUtils.primitiveToWrapper(f.getType());
+      PojoUtils.Setter setter = PojoUtils.createSetter(outputClass, field, c);
+
+      // Generate evaluated expression
+
+      Expression expression = PojoUtils
+          .createExpression(inputClass, expr, c, expressionFuntions.toArray(new String[expressionFuntions.size()]));
+
+      transformationMap.put(setter, expression);
+    }
+  }
+
+  @Override
+  public void deactivate()
+  {
+  }
+
+  /**
+   * Returns expression map which defines outputFieldName => Expression mapping.
+   *
+   * @return Map of outputFieldName => Expression
+   */
+  public Map<String, String> getExpressionMap()
+  {
+    return expressionMap;
+  }
+
+  /**
+   * Set expression map (outputFieldName => Expression) which defines how output POJO should be generated.
+   * This is a mandatory property.
+   * @param expressionMap Map of String => String defining expression for output field.
+   *
+   * @description $(key) Output field for which expression should be evaluated
+   * @description $(value) Expression to be evaluated for output field.
+   * @useSchema $(key) input.fields[].name
+   */
+  public void setExpressionMap(Map<String, String> expressionMap)
+  {
+    this.expressionMap = expressionMap;
+  }
+
+  /**
+   * Returns the list of expression function which would be made available to expression to use.
+   *
+   * @return List of function that are available in expression.
+   */
+  public List<String> getExpressionFuntions()
+  {
+    return expressionFuntions;
+  }
+
+  /**
+   * Set list of import classes/method should should be made statically available to expression to use.
+   * For ex. org.apache.apex.test1.Test would mean that "Test" method will be available in the expression to be
+   * used directly.
+   * This is an optional property. See constructor to see defaults that are included.
+   *
+   * @param expressionFuntions List of qualified class/method that needs to be imported to expression.
+   */
+  public void setExpressionFuntions(List<String> expressionFuntions)
+  {
+    this.expressionFuntions = expressionFuntions;
+  }
+
+  /**
+   * Returns expression to be evaluated for given output field.
+   * This method is meant to be used for properties.xml file where map should be set using BeanUtils constructs.
+   *
+   * @param field Field for which expression needs to be evaluated.
+   * @return Expression that will be evaluated for given field.
+   * @omitFromUI
+   */
+  public String getExpressionMap(String field)
+  {
+    return (this.expressionMap != null) ? this.expressionMap.get(field) : null;
+  }
+
+  /**
+   * Set expression for given field.
+   * This is a mandatory property.
+   * This method is meant to be used for properties.xml file where map should be set using BeanUtils constructs.
+   *
+   * @param field Output field for which expression should be set.
+   * @param expression Expression that should be evaluated for given output field.
+   * @omitFromUI
+   */
+  public void setExpressionMap(String field, String expression)
+  {
+    if (this.expressionMap == null) {
+      this.expressionMap = new HashMap<>();
+    }
+    this.expressionMap.put(field, expression);
+  }
+
+  /**
+   * Return expression function for given index.
+   * This method is meant to be used for setting indexed property using BeanUtils constructs.
+   *
+   * @param index Index at which expression function should be returned.
+   * @return Qualified Expression function at given index.
+   * @omitFromUI
+   */
+  public String getExpressionFunctions(int index)
+  {
+    return (index < this.expressionFuntions.size()) ? this.expressionFuntions.get(index) : null;
+  }
+
+  /**
+   * Sets expression function are given index.
+   * This method is meant to be used for setting indexed property using BeanUtils constructs.
+   *
+   * @param index Index at which expression should be set.
+   * @param expressionFunction Qualified expression function that should be made available to expression.
+   * @omitFromUI
+   */
+  public void setExpressionFunctions(int index, String expressionFunction)
+  {
+    for (int i = this.expressionFuntions.size(); i <= index; i++) {
+      this.expressionFuntions.add(null);
+    }
+    this.expressionFuntions.set(index, expressionFunction);
+  }
+
+  /**
+   * Tells whether the matching (by name and by type) fields between input and output should be copied as is.
+   *
+   * @return Tells whether the matching (by name and by type) fields between input and output should be copied as is.
+   */
+  public boolean isCopyMatchingFields()
+  {
+    return copyMatchingFields;
+  }
+
+  /**
+   * Set whether the matching (by name and by type) fields between input and output should be copied as is.
+   * This is an optional property, default is true.
+   *
+   * @param copyMatchingFields true/false
+   */
+  public void setCopyMatchingFields(boolean copyMatchingFields)
+  {
+    this.copyMatchingFields = copyMatchingFields;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/47df5ebc/library/src/main/java/com/datatorrent/lib/util/PojoUtils.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/com/datatorrent/lib/util/PojoUtils.java b/library/src/main/java/com/datatorrent/lib/util/PojoUtils.java
index fd33675..c4b3daa 100644
--- a/library/src/main/java/com/datatorrent/lib/util/PojoUtils.java
+++ b/library/src/main/java/com/datatorrent/lib/util/PojoUtils.java
@@ -18,23 +18,24 @@
  */
 package com.datatorrent.lib.util;
 
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
+import com.datatorrent.lib.expression.Expression;
+import com.datatorrent.lib.expression.JavaExpressionParser;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
-
+import org.apache.commons.lang3.ClassUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.codehaus.commons.compiler.CompileException;
 import org.codehaus.commons.compiler.CompilerFactoryFactory;
 import org.codehaus.commons.compiler.IScriptEvaluator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.commons.lang3.ClassUtils;
-import org.apache.commons.lang3.StringUtils;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
 
 /**
  * @since 2.1.0
@@ -45,6 +46,7 @@ public class PojoUtils
 
   public static final String DEFAULT_EXP_OBJECT_PLACEHOLDER = "{$}";
   public static final String DEFAULT_EXP_VAL_PLACEHOLDER = "{#}";
+  public static final String DEFAULT_EXPRESSION_OBJ_PLACEHOLDER = "$";
 
   private static final String OBJECT = "object";
   private static final String VAL = "val";
@@ -572,41 +574,13 @@ public class PojoUtils
   @SuppressWarnings("StringEquality")
   private static Object createGetter(Class<?> pojoClass, String getterExpr, String exprObjectPlaceholder, Class<?> exprClass, Class<?> getterClass)
   {
-    if (getterExpr.startsWith(".")) {
-      getterExpr = getterExpr.substring(1);
-    }
-
-    if (getterExpr.isEmpty()) {
-      throw new IllegalArgumentException("The getter expression: \"" + getterExpr + "\" is invalid.");
-    }
-
     logger.debug("{} {} {} {}", pojoClass, getterExpr, exprClass, getterClass);
 
-    IScriptEvaluator se;
+    JavaExpressionParser javaExpressionParser = new JavaExpressionParser();
+    javaExpressionParser.setInputObjectPlaceholder(PojoUtils.DEFAULT_EXPRESSION_OBJ_PLACEHOLDER, PojoUtils.OBJECT);
+    String code = javaExpressionParser.convertToCompilableExpression(getterExpr, pojoClass, exprClass);
 
-    try {
-      se = CompilerFactoryFactory.getDefaultCompilerFactory().newScriptEvaluator();
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-
-    String code = StringUtils.replaceEach(getterExpr, new String[]{exprObjectPlaceholder},
-            new String[]{new JavaStatement(pojoClass.getName().length() + OBJECT.length() + 4).appendCastToTypeExpr(pojoClass, OBJECT).toString()});
-    if (code != getterExpr) {
-      code = new JavaReturnStatement(exprClass.getName().length() + code.length() + 12, exprClass).append(code).getStatement();
-      logger.debug("Original expression {} is a complex expression. Replacing it with {}.", getterExpr, code);
-    }
-    else {
-      code = getSingleFieldGetterExpression(pojoClass, getterExpr, exprClass);
-    }
-
-    logger.debug("code: {}", code);
-
-    try {
-      return se.createFastEvaluator(code, getterClass, new String[] {PojoUtils.OBJECT});
-    } catch (CompileException ex) {
-      throw new RuntimeException(ex);
-    }
+    return compileExpression(code, getterClass, new String[] {PojoUtils.OBJECT});
   }
 
   private static String getSingleFieldSetterExpression(final Class<?> pojoClass, final String fieldExpression, final Class<?> exprClass)
@@ -677,13 +651,6 @@ public class PojoUtils
 
     logger.debug("{} {} {} {}", pojoClass, setterExpr, exprClass, setterClass);
 
-    IScriptEvaluator se;
-
-    try {
-      se = CompilerFactoryFactory.getDefaultCompilerFactory().newScriptEvaluator();
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
 
 
     String code = StringUtils.replaceEach(setterExpr, new String[]{exprObjectPlaceholder, exprValPlaceholder},
@@ -696,11 +663,99 @@ public class PojoUtils
       code = getSingleFieldSetterExpression(pojoClass, setterExpr, exprClass);
     }
 
+    return compileExpression(code, setterClass, new String[] {PojoUtils.OBJECT, PojoUtils.VAL});
+  }
+
+  /**
+   * This method takes in expression, compiles the expression to provide a executable form of expression.
+   * This method uses {@link com.datatorrent.lib.expression.JavaExpressionParser} as expression parser.
+   *
+   * @param inputType  Type of input object
+   * @param expr       expression to be compiled.
+   * @param returnType Return type of the expression.
+   * @return Object of type {@link Expression} which can be directly executed.
+   */
+  public static Expression createExpression(Class<?> inputType, String expr, Class<?> returnType)
+  {
+    return createExpression(inputType, expr, returnType, null);
+  }
+
+  /**
+   * This method takes in expression, compiles the expression to provide a executable form of expression.
+   * This methods also takes in list of classes and method which can be imported statically in expression.
+   * <p/>
+   * This method uses {@link JavaExpressionParser} as expression parser.
+   *
+   * @param inputType      Type of input object
+   * @param expr           expression to be compiled.
+   * @param returnType     Return type of the expression.
+   * @param defaultImports List of classes/method which will be statically imported to expression compilation.
+   * @return Object of type {@link Expression} which can be directly executed.
+   */
+  public static Expression createExpression(Class<?> inputType, String expr, Class<?> returnType,
+      String[] defaultImports)
+  {
+    JavaExpressionParser javaExpressionParser = new JavaExpressionParser();
+    javaExpressionParser.setInputObjectPlaceholder("$", PojoUtils.OBJECT);
+
+    return createExpression(inputType, expr, returnType, defaultImports, javaExpressionParser);
+  }
+
+  /**
+   * This method takes in expression, compiles the expression to provide a executable form of expression.
+   * This methods also takes in list of classes and method which can be imported statically in expression.
+   * <p/>
+   * Using this method one can override expression parser implementation.
+   *
+   * @param inputType      Type of input object
+   * @param expr           expression to be compiled.
+   * @param returnType     Return type of the expression.
+   * @param defaultImports List of classes/method which will be statically imported to expression compilation.
+   * @param parser         Expression parser that should be used to parse expression.
+   * @return Object of type {@link Expression} which can be directly executed.
+   * @see {@link JavaExpressionParser} as a example.
+   */
+  public static Expression createExpression(Class<?> inputType, String expr, Class<?> returnType,
+      String[] defaultImports, Expression.ExpressionParser parser)
+  {
+    String code = parser.convertToCompilableExpression(expr, inputType, returnType);
+
+    return (Expression)compileExpression(code, Expression.class, new String[] {PojoUtils.OBJECT}, defaultImports);
+  }
+
+  private static Object compileExpression(String code, Class<?> implClass, String[] params)
+  {
+    return compileExpression(code, implClass, params, null);
+  }
+
+  private static Object compileExpression(String code, Class<?> implClass, String[] params, String[] defaultImports)
+  {
+    List<String> imports = new LinkedList<>();
+    if (defaultImports != null && defaultImports.length != 0) {
+      for (String defaultImport : defaultImports) {
+        if (defaultImport != null) {
+          if (!defaultImport.startsWith("static")) {
+            imports.add("static " + defaultImport);
+          } else {
+            imports.add(defaultImport);
+          }
+        }
+      }
+    }
+
+    IScriptEvaluator se;
+
     try {
+      se = CompilerFactoryFactory.getDefaultCompilerFactory().newScriptEvaluator();
+      se.setDefaultImports(imports.toArray(new String[imports.size()]));
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
 
+    try {
       logger.debug("code: {}", code);
 
-      return se.createFastEvaluator(code, setterClass, new String[] { PojoUtils.OBJECT, PojoUtils.VAL});
+      return se.createFastEvaluator(code, implClass, params);
     } catch (CompileException ex) {
       throw new RuntimeException(ex);
     }

http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/47df5ebc/library/src/test/java/com/datatorrent/lib/transform/TransformOperatorAppTest.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/com/datatorrent/lib/transform/TransformOperatorAppTest.java b/library/src/test/java/com/datatorrent/lib/transform/TransformOperatorAppTest.java
new file mode 100644
index 0000000..fc8040b
--- /dev/null
+++ b/library/src/test/java/com/datatorrent/lib/transform/TransformOperatorAppTest.java
@@ -0,0 +1,130 @@
+/**
+ * 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 com.datatorrent.lib.transform;
+
+import java.util.HashMap;
+import java.util.Map;
+import javax.validation.ConstraintViolationException;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.Context;
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.InputOperator;
+import com.datatorrent.api.LocalMode;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.lib.io.ConsoleOutputOperator;
+
+/**
+ * Application Test for Transform Operator.
+ */
+public class TransformOperatorAppTest
+{
+  @Test
+  public void testExpressionApplication() throws Exception
+  {
+    try {
+      LocalMode lma = LocalMode.newInstance();
+      Configuration conf = new Configuration(false);
+      lma.prepareDAG(new Application(), conf);
+      LocalMode.Controller lc = lma.getController();
+      lc.run(10000); // runs for 10 seconds and quits
+    } catch (ConstraintViolationException e) {
+      Assert.fail("constraint violations: " + e.getConstraintViolations());
+    }
+  }
+
+  public static class Application implements StreamingApplication
+  {
+    @Override public void populateDAG(DAG dag, Configuration configuration)
+    {
+      DummyInputGenerator input = dag.addOperator("Input", new DummyInputGenerator());
+      TransformOperator transform = dag.addOperator("Transform", new TransformOperator());
+
+      Map<String, String> expMap = new HashMap<>();
+      expMap.put("firstname", "lastname");
+      expMap.put("lastname", "firstname");
+      transform.setExpressionMap(expMap);
+
+      ConsoleOutputOperator console = dag.addOperator("Console", new ConsoleOutputOperator());
+      console.setSilent(true);
+      dag.getMeta(transform).getMeta(transform.output).getAttributes().put(Context.PortContext.TUPLE_CLASS, TestPojo.class);
+      dag.getMeta(transform).getMeta(transform.input).getAttributes().put(Context.PortContext.TUPLE_CLASS, TestPojo.class);
+
+      dag.addStream("Connect", input.output, transform.input);
+      dag.addStream("Ppint", transform.output, console.input);
+    }
+  }
+
+  public static class TestPojo
+  {
+    private String firstname;
+    public String lastname;
+
+    public TestPojo()
+    {
+      //for kryo
+    }
+
+    public TestPojo(String firstname, String lastname)
+    {
+      this.firstname = firstname;
+      this.lastname = lastname;
+    }
+
+    public String getFirstname()
+    {
+      return firstname;
+    }
+
+    public void setFirstname(String firstname)
+    {
+      this.firstname = firstname;
+    }
+  }
+
+  public static class DummyInputGenerator implements InputOperator
+  {
+    public final transient DefaultOutputPort<TestPojo> output = new DefaultOutputPort<>();
+
+    @Override public void emitTuples()
+    {
+      output.emit(new TestPojo("FirstName", "LastName"));
+    }
+
+    @Override public void beginWindow(long l)
+    {
+    }
+
+    @Override public void endWindow()
+    {
+    }
+
+    @Override public void setup(Context.OperatorContext context)
+    {
+    }
+
+    @Override public void teardown()
+    {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/47df5ebc/library/src/test/java/com/datatorrent/lib/transform/TransformOperatorTest.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/com/datatorrent/lib/transform/TransformOperatorTest.java b/library/src/test/java/com/datatorrent/lib/transform/TransformOperatorTest.java
new file mode 100644
index 0000000..1fd8344
--- /dev/null
+++ b/library/src/test/java/com/datatorrent/lib/transform/TransformOperatorTest.java
@@ -0,0 +1,356 @@
+/**
+ * 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 com.datatorrent.lib.transform;
+
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.Description;
+
+import com.datatorrent.api.Attribute;
+import com.datatorrent.api.Context;
+import com.datatorrent.lib.testbench.CollectorTestSink;
+import com.datatorrent.lib.util.TestUtils;
+import com.datatorrent.stram.engine.PortContext;
+
+/**
+ * Test for Transform Operator.
+ */
+public class TransformOperatorTest
+{
+  private TransformOperator operator;
+  CollectorTestSink<Object> sink;
+
+  @Rule
+  public TestUtils.TestInfo testMeta = new TestUtils.TestInfo()
+  {
+    @Override
+    protected void starting(Description description)
+    {
+      super.starting(description);
+      operator = new TransformOperator();
+
+      sink = new CollectorTestSink<>();
+      TestUtils.setSink(operator.output, sink);
+
+      operator.setup(null);
+
+      Attribute.AttributeMap inMap = new Attribute.AttributeMap.DefaultAttributeMap();
+      inMap.put(Context.PortContext.TUPLE_CLASS, InputClass.class);
+      operator.input.setup(new PortContext(inMap, null));
+
+      Attribute.AttributeMap outMap = new Attribute.AttributeMap.DefaultAttributeMap();
+      outMap.put(Context.PortContext.TUPLE_CLASS, OutputClass.class);
+      operator.output.setup(new PortContext(outMap, null));
+    }
+
+    @Override
+    protected void finished(Description description)
+    {
+      operator.deactivate();
+      operator.teardown();
+      super.finished(description);
+    }
+  };
+
+  @Test
+  public void testTransformOperator()
+  {
+    // Set expression map
+    Map<String, String> expMap = new HashMap<>();
+    expMap.put("var21", "var1");
+    expMap.put("var22", "$.var2");
+    expMap.put("var23", "{$.var3}");
+    operator.setExpressionMap(expMap);
+    operator.activate(null);
+
+    operator.beginWindow(1L);
+    InputClass inputClass = new InputClass();
+    inputClass.setVar1(123);
+    inputClass.setVar2(12);
+    inputClass.setVar3("ABC");
+    operator.input.put(inputClass);
+    operator.endWindow();
+
+    Assert.assertEquals(1, sink.collectedTuples.size());
+    Object o = sink.collectedTuples.get(0);
+    Assert.assertTrue(o instanceof OutputClass);
+    OutputClass out = (OutputClass)o;
+    Assert.assertEquals(123, out.getVar21());
+    Assert.assertEquals(12, out.getVar22());
+    Assert.assertEquals("ABC", out.getVar23());
+  }
+
+  @Test
+  public void testComplexTransformOperator()
+  {
+    // Set expression map
+    Map<String, String> expMap = new HashMap<>();
+    expMap.put("var21", "{$.var1} * 123");
+    expMap.put("var22", "round(pow({$.var2}, {$.var1}/50))");
+    expMap.put("var23", "{$.var3}.toLowerCase()");
+    operator.setExpressionMap(expMap);
+    operator.activate(null);
+
+    operator.beginWindow(1L);
+    InputClass inputClass = new InputClass();
+    inputClass.setVar1(123);
+    inputClass.setVar2(12);
+    inputClass.setVar3("ABC");
+    operator.input.put(inputClass);
+    operator.endWindow();
+
+    Assert.assertEquals(1, sink.collectedTuples.size());
+    Object o = sink.collectedTuples.get(0);
+    Assert.assertTrue(o instanceof OutputClass);
+    OutputClass out = (OutputClass)o;
+    Assert.assertEquals(15129, out.getVar21());
+    Assert.assertEquals(144, out.getVar22());
+    Assert.assertEquals("abc", out.getVar23());
+  }
+
+  @Test
+  public void testCopyFieldOperator()
+  {
+    operator.setCopyMatchingFields(true);
+
+    // Set expression map
+    Map<String, String> expMap = new HashMap<>();
+    expMap.put("var21", "{$.var1} * 123");
+    expMap.put("var22", "round(pow({$.var2}, {$.var1}/50))");
+    expMap.put("var23", "{$.var3}.toLowerCase()");
+    expMap.put("var4", "{$.var4}.toLowerCase()");
+    operator.setExpressionMap(expMap);
+    operator.activate(null);
+
+    operator.beginWindow(1L);
+    InputClass inputClass = new InputClass();
+    inputClass.setVar1(123);
+    inputClass.setVar2(12);
+    inputClass.setVar3("ABC");
+    inputClass.setVar4("XYZ");
+    inputClass.setVar5(12345);
+    inputClass.setVar6(123);
+    inputClass.setVar7(456);
+    inputClass.setVar9(789);
+    operator.input.put(inputClass);
+    operator.endWindow();
+
+    Assert.assertEquals(1, sink.collectedTuples.size());
+    Object o = sink.collectedTuples.get(0);
+    Assert.assertTrue(o instanceof OutputClass);
+    OutputClass out = (OutputClass)o;
+    Assert.assertEquals(15129, out.getVar21());
+    Assert.assertEquals(144, out.getVar22());
+    Assert.assertEquals("abc", out.getVar23());
+    Assert.assertEquals("xyz", out.getVar4());
+    Assert.assertEquals(12345, out.getVar5());
+    Assert.assertEquals(123, out.getVar6());
+    Assert.assertEquals(0, out.getVar7());
+    Assert.assertEquals(0, out.getVar8());
+  }
+
+  public static class InputClass
+  {
+    private int var1;
+    public long var2;
+    private String var3;
+    private String var4;
+    public int var5;
+    private long var6;
+    public int var7;
+    public long var9;
+
+    public int getVar1()
+    {
+      return var1;
+    }
+
+    public void setVar1(int var1)
+    {
+      this.var1 = var1;
+    }
+
+    public long getVar2()
+    {
+      return var2;
+    }
+
+    public void setVar2(long var2)
+    {
+      this.var2 = var2;
+    }
+
+    public String getVar3()
+    {
+      return var3;
+    }
+
+    public void setVar3(String var3)
+    {
+      this.var3 = var3;
+    }
+
+    public String getVar4()
+    {
+      return var4;
+    }
+
+    public void setVar4(String var4)
+    {
+      this.var4 = var4;
+    }
+
+    public int getVar5()
+    {
+      return var5;
+    }
+
+    public void setVar5(int var5)
+    {
+      this.var5 = var5;
+    }
+
+    public long getVar6()
+    {
+      return var6;
+    }
+
+    public void setVar6(long var6)
+    {
+      this.var6 = var6;
+    }
+
+    public int getVar7()
+    {
+      return var7;
+    }
+
+    public void setVar7(int var7)
+    {
+      this.var7 = var7;
+    }
+
+    public long getVar9()
+    {
+      return var9;
+    }
+
+    public void setVar9(long var9)
+    {
+      this.var9 = var9;
+    }
+  }
+
+  public static class OutputClass
+  {
+    private int var21;
+    public long var22;
+    private String var23;
+    private String var4;
+    public int var5;
+    public long var6;
+    public long var7;
+    private long var8;
+
+    public int getVar21()
+    {
+      return var21;
+    }
+
+    public void setVar21(int var21)
+    {
+      this.var21 = var21;
+    }
+
+    public long getVar22()
+    {
+      return var22;
+    }
+
+    public void setVar22(long var22)
+    {
+      this.var22 = var22;
+    }
+
+    public String getVar23()
+    {
+      return var23;
+    }
+
+    public void setVar23(String var23)
+    {
+      this.var23 = var23;
+    }
+
+    public String getVar4()
+    {
+      return var4;
+    }
+
+    public void setVar4(String var4)
+    {
+      this.var4 = var4;
+    }
+
+    public int getVar5()
+    {
+      return var5;
+    }
+
+    public void setVar5(int var5)
+    {
+      this.var5 = var5;
+    }
+
+    public long getVar6()
+    {
+      return var6;
+    }
+
+    public void setVar6(long var6)
+    {
+      this.var6 = var6;
+    }
+
+    public long getVar7()
+    {
+      return var7;
+    }
+
+    public void setVar7(long var7)
+    {
+      this.var7 = var7;
+    }
+
+    public long getVar8()
+    {
+      return var8;
+    }
+
+    public void setVar8(long var8)
+    {
+      this.var8 = var8;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/47df5ebc/library/src/test/java/com/datatorrent/lib/util/PojoUtilsTest.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/com/datatorrent/lib/util/PojoUtilsTest.java b/library/src/test/java/com/datatorrent/lib/util/PojoUtilsTest.java
index 5b1b3f1..138c731 100644
--- a/library/src/test/java/com/datatorrent/lib/util/PojoUtilsTest.java
+++ b/library/src/test/java/com/datatorrent/lib/util/PojoUtilsTest.java
@@ -20,6 +20,7 @@ package com.datatorrent.lib.util;
 
 import static com.datatorrent.lib.util.PojoUtils.constructGetter;
 import static com.datatorrent.lib.util.PojoUtils.constructSetter;
+import static com.datatorrent.lib.util.PojoUtils.createExpression;
 import static com.datatorrent.lib.util.PojoUtils.createGetter;
 import static com.datatorrent.lib.util.PojoUtils.createGetterBoolean;
 import static com.datatorrent.lib.util.PojoUtils.createGetterByte;
@@ -39,12 +40,15 @@ import static com.datatorrent.lib.util.PojoUtils.createSetterInt;
 import static com.datatorrent.lib.util.PojoUtils.createSetterLong;
 import static com.datatorrent.lib.util.PojoUtils.createSetterShort;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.datatorrent.lib.expression.Expression;
 import com.datatorrent.lib.util.PojoUtils.GetterBoolean;
 import com.datatorrent.lib.util.PojoUtils.GetterByte;
 import com.datatorrent.lib.util.PojoUtils.GetterChar;
@@ -73,7 +77,7 @@ public class PojoUtilsTest
   @Test
   public void testGetters() throws Exception
   {
-    /* let mvn know that janino is dynamically loaded jar */
+//    /* let mvn know that janino is dynamically loaded jar */
     Assert.assertNotNull(org.codehaus.janino.util.AutoIndentWriter.class);
 
     GetterBoolean<Object> getBoolean = createGetterBoolean(fqcn, "innerObj.boolVal");
@@ -105,7 +109,6 @@ public class PojoUtilsTest
 
     Getter<Object, Object> getObject = createGetter(fqcn, "innerObj.objVal", Object.class);
     assertEquals(testObj.innerObj.getObjVal(), getObject.get(testObj));
-
   }
 
   @Test
@@ -306,7 +309,6 @@ public class PojoUtilsTest
     {
       throw new UnsupportedOperationException("not the right method");
     }
-
   }
 
   @Test
@@ -461,14 +463,6 @@ public class PojoUtilsTest
   }
 
   @Test (expected = RuntimeException.class)
-  public void testPrivateField()
-  {
-    final Class<?> testPojoClass = TestPojo.class;
-    @SuppressWarnings("unused")
-    SetterInt<Object> setterInt = createSetterInt(testPojoClass, "privateIntField");
-  }
-
-  @Test (expected = RuntimeException.class)
   public void testWrongSetterMethod()
   {
     final Class<?> testPojoClass = TestPojo.class;
@@ -489,4 +483,91 @@ public class PojoUtilsTest
     final Class<?> testPojoClass = TestPojo.class;
     createSetter(testPojoClass, TestPojo.INT_FIELD_NAME, int.class);
   }
+
+  @Test (expected = RuntimeException.class)
+  public void testPrivateFieldExpression()
+  {
+    final Class<?> testPojoClass = TestPojo.class;
+    createExpression(testPojoClass, "privateIntField", int.class);
+  }
+
+  @Test
+  public void testBasicExpression()
+  {
+    TestPojo testObj = new TestPojo(1);
+    Class<?> testObjClass = testObj.getClass();
+
+    assertEquals(testObj.getIntVal(), createExpression(testObjClass, "intField", int.class).execute(testObj));
+    assertEquals(testObj.getIntVal(), createExpression(testObjClass, "intVal", int.class).execute(testObj));
+  }
+
+  @Test (expected = RuntimeException.class)
+  public void testPrivateField()
+  {
+    final Class<?> testPojoClass = TestPojo.class;
+    @SuppressWarnings("unused")
+    SetterInt<Object> setterInt = createSetterInt(testPojoClass, "privateIntField");
+  }
+
+  @Test
+  public void testNestedPOJOExpression()
+  {
+    // Evaluate and execute expression for simple inner boolean value expressed as similar to PojoUtils Getter.
+    Expression expression = createExpression(fqcn, "innerObj.boolVal", boolean.class);
+    assertTrue((Boolean)expression.execute(testObj));
+
+    // Evaluate and execute expression for simple inner boolean value expressed with expression syntax.
+    expression = createExpression(fqcn, "{$.innerObj.boolVal}", boolean.class);
+    assertTrue((Boolean)expression.execute(testObj));
+
+    // Evaluate and execute expression for simple inner boolean value expressed with expression syntax where there is no compiler hint.
+    expression = createExpression(fqcn, "$.innerObj.boolVal", boolean.class);
+    assertTrue((Boolean)expression.execute(testObj));
+
+    // Evaluate and execute expression for simple inner boolean value and not the returned value.
+    expression = createExpression(fqcn, "!{$.innerObj.boolVal}", boolean.class);
+    assertFalse((Boolean)expression.execute(testObj));
+
+    // Access the object with compiler hint and then innerObj and boolVal are publicly accessible method.
+    expression = createExpression(fqcn, "!{$}.innerObj.boolVal", boolean.class);
+    assertFalse((Boolean)expression.execute(testObj));
+  }
+
+  @Test
+  public void testComplexPOJOExpression()
+  {
+    Expression expression = createExpression(fqcn, "{$}.innerObj.boolVal && ({$}.innerObj.intVal == 11)", boolean.class);
+    assertTrue((Boolean)expression.execute(testObj));
+
+    expression = createExpression(fqcn, "valueOf({$.innerObj.privateIntVal}) + substring({$.innerObj.stringVal}, 2) + {$.innerObj.privateStringVal}.length()",
+        String.class, new String[]{"org.apache.commons.lang3.StringUtils.*",
+            "java.lang.String.valueOf"});
+    assertEquals("11llo5", expression.execute(testObj));
+
+    expression = createExpression(fqcn, "round(pow({$.innerObj.privateFloatVal}, {$.innerObj.privateDoubleVal}))", long.class, new String[] {"java.lang.Math.*"});
+    assertEquals(46162L, expression.execute(testObj));
+  }
+
+  @Test
+  public void testExpressionSerialization()
+  {
+    Expression expression = createExpression(fqcn, "{$.innerObj.boolVal}", boolean.class);
+    assertTrue((Boolean)expression.execute(testObj));
+  }
+
+  @Test
+  public void testCustomImports()
+  {
+    Expression expression = createExpression(fqcn, "concat({$.innerObj.stringVal}, {$.innerObj.privateStringVal})",
+        String.class, new String[] {TestImports.class.getName() + ".concat"});
+    assertEquals("hello hello", expression.execute(testObj));
+  }
+
+  public static class TestImports
+  {
+    public static String concat(String a, String b)
+    {
+      return a + " " + b;
+    }
+  }
 }


[2/3] incubator-apex-malhar git commit: Merge branch 'APEXMALHAR-2010_Transform' of https://github.com/chinmaykolhatkar/incubator-apex-malhar

Posted by th...@apache.org.
Merge branch 'APEXMALHAR-2010_Transform' of https://github.com/chinmaykolhatkar/incubator-apex-malhar


Project: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/commit/264f629c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/tree/264f629c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/diff/264f629c

Branch: refs/heads/devel-3
Commit: 264f629c13a6a93fadaba367ac9404bfa0edecee
Parents: 5373a3c 47df5eb
Author: Sandeep Deshmukh <sa...@datatorrent.com>
Authored: Wed Mar 23 11:25:38 2016 +0530
Committer: Sandeep Deshmukh <sa...@datatorrent.com>
Committed: Wed Mar 23 11:25:38 2016 +0530

----------------------------------------------------------------------
 .../datatorrent/lib/expression/Expression.java  |  61 ++++
 .../lib/expression/JavaExpressionParser.java    | 159 +++++++++
 .../lib/transform/TransformOperator.java        | 296 +++++++++++++++
 .../com/datatorrent/lib/util/PojoUtils.java     | 153 +++++---
 .../lib/transform/TransformOperatorAppTest.java | 130 +++++++
 .../lib/transform/TransformOperatorTest.java    | 356 +++++++++++++++++++
 .../com/datatorrent/lib/util/PojoUtilsTest.java | 103 +++++-
 7 files changed, 1198 insertions(+), 60 deletions(-)
----------------------------------------------------------------------



[3/3] incubator-apex-malhar git commit: Merge branch 'master' into devel-3

Posted by th...@apache.org.
Merge branch 'master' into devel-3


Project: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/commit/d9abee96
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/tree/d9abee96
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/diff/d9abee96

Branch: refs/heads/devel-3
Commit: d9abee962ae2d7d079852daea8a56b030ff398b9
Parents: a8fbcac 264f629
Author: Thomas Weise <th...@datatorrent.com>
Authored: Fri Mar 25 06:16:26 2016 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Fri Mar 25 06:16:26 2016 -0700

----------------------------------------------------------------------
 .../datatorrent/lib/expression/Expression.java  |  61 ++++
 .../lib/expression/JavaExpressionParser.java    | 159 +++++++++
 .../lib/transform/TransformOperator.java        | 296 +++++++++++++++
 .../com/datatorrent/lib/util/PojoUtils.java     | 153 +++++---
 .../lib/transform/TransformOperatorAppTest.java | 130 +++++++
 .../lib/transform/TransformOperatorTest.java    | 356 +++++++++++++++++++
 .../com/datatorrent/lib/util/PojoUtilsTest.java | 103 +++++-
 7 files changed, 1198 insertions(+), 60 deletions(-)
----------------------------------------------------------------------