Uploaded image for project: 'Phoenix'
  1. Phoenix
  2. PHOENIX-1978

UDF ArgumentTypeMismatchException

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 4.4.1
    • None
    • None

    Description

      I created a sample UDF with Phoenix 4.4 by copying ARRAY_ELEM function (https://github.com/apache/phoenix/blob/master/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayIndexFunction.java) and removing the BuiltInFunction annotation

      after deploying to the server and running the following queries it throws an exception below

      create function UDF_ARRAY_ELEM(VARCHAR ARRAY, INTEGER) returns "VARCHAR" as 'com.test.ArrayIndexFunction' using jar '/tmp/udf.jar'

      CREATE TABLE TESTTABLE10(ID VARCHAR NOT NULL, NAME VARCHAR ARRAY, CITY VARCHAR ARRAY CONSTRAINT pk PRIMARY KEY (ID) )

      UPSERT INTO TESTTABLE10(ID,NAME,CITY) VALUES('111', ARRAY['JOHN','MIKE','BOB'], ARRAY['NYC','LA','SF'])

      UPSERT INTO TESTTABLE10(ID,NAME,CITY) VALUES('112', ARRAY['CHEN','CARL','ALICE'], ARRAY['BOSTON','WASHINGTON','PALO ALTO'])

      SELECT ID, UDF_ARRAY_ELEM(NAME, 2) FROM TESTTABLE10

      org.apache.phoenix.schema.ArgumentTypeMismatchException: ERROR 203 (22005): Type mismatch. expected: [INTEGER ARRAY] but was: INTEGER at UDF_ARRAY_ELEM argument 2
      at org.apache.phoenix.parse.FunctionParseNode.validate(FunctionParseNode.java:195)
      at org.apache.phoenix.compile.ExpressionCompiler.visitLeave(ExpressionCompiler.java:326)
      at org.apache.phoenix.compile.ProjectionCompiler$SelectClauseVisitor.visitLeave(ProjectionCompiler.java:637)
      at org.apache.phoenix.compile.ProjectionCompiler$SelectClauseVisitor.visitLeave(ProjectionCompiler.java:538)
      at org.apache.phoenix.parse.FunctionParseNode.accept(FunctionParseNode.java:85)Hbase>
      at org.apache.phoenix.compile.ProjectionCompiler.compile(ProjectionCompiler.java:396)
      at org.apache.phoenix.compile.QueryCompiler.compileSingleFlatQuery(QueryCompiler.java:537)
      at org.apache.phoenix.compile.QueryCompiler.compileSingleQuery(QueryCompiler.java:488)
      at org.apache.phoenix.compile.QueryCompiler.compileSelect(QueryCompiler.java:200)
      at org.apache.phoenix.compile.QueryCompiler.compile(QueryCompiler.java:157)
      at org.apache.phoenix.jdbc.PhoenixStatement$ExecutableSelectStatement.compilePlan(PhoenixStatement.java:361)
      at org.apache.phoenix.jdbc.PhoenixStatement$ExecutableSelectStatement.compilePlan(PhoenixStatement.java:335)
      at org.apache.phoenix.jdbc.PhoenixStatement$1.call(PhoenixStatement.java:243)
      at org.apache.phoenix.jdbc.PhoenixStatement$1.call(PhoenixStatement.java:238)
      at org.apache.phoenix.call.CallRunner.run(CallRunner.java:53)
      at org.apache.phoenix.jdbc.PhoenixStatement.executeQuery(PhoenixStatement.java:237)
      at org.apache.phoenix.jdbc.PhoenixPreparedStatement.executeQuery(PhoenixPreparedStatement.java:187)
      at CustomSQL.openTerminal(CustomSQL.java:45)
      at CustomSQL.main(CustomSQL.java:19)

      udf function:

      package com.test;

      import java.util.List;

      import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
      import org.apache.phoenix.expression.Expression;
      import org.apache.phoenix.expression.function.ScalarFunction;
      import org.apache.phoenix.parse.FunctionParseNode.Argument;
      import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
      import org.apache.phoenix.parse.ParseException;
      import org.apache.phoenix.schema.types.PBinaryArray;
      import org.apache.phoenix.schema.types.PInteger;
      import org.apache.phoenix.schema.types.PArrayDataType;
      import org.apache.phoenix.schema.types.PDataType;
      import org.apache.phoenix.schema.types.PVarbinaryArray;
      import org.apache.phoenix.schema.SortOrder;
      import org.apache.phoenix.schema.tuple.Tuple;

      /*@BuiltInFunction(name = ArrayIndexFunction.NAME, args = {
      @Argument(allowedTypes =

      { PBinaryArray.class, PVarbinaryArray.class }

      ),
      @Argument(allowedTypes =

      { PInteger.class }

      ) })
      */
      public class ArrayIndexFunction extends ScalarFunction {

      public static final String NAME = "UDF_ARRAY_ELEM";

      public ArrayIndexFunction() {
      }

      public ArrayIndexFunction(List<Expression> children)

      { super(children); }

      @Override
      public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
      Expression indexExpr = children.get(1);
      if (!indexExpr.evaluate(tuple, ptr))

      { return false; }

      else if (ptr.getLength() == 0)

      { return true; }

      // Use Codec to prevent Integer object allocation
      int index = PInteger.INSTANCE.getCodec().decodeInt(ptr, indexExpr.getSortOrder());
      if(index < 0)

      { throw new ParseException("Index cannot be negative :" + index); }

      Expression arrayExpr = children.get(0);
      return PArrayDataType.positionAtArrayElement(tuple, ptr, index, arrayExpr, getDataType(), getMaxLength());
      }

      @Override
      public PDataType getDataType()

      { return PDataType.fromTypeId(children.get(0).getDataType().getSqlType() - PDataType.ARRAY_TYPE_BASE); }

      @Override
      public Integer getMaxLength()

      { return this.children.get(0).getMaxLength(); }

      @Override
      public String getName()

      { return NAME; }

      @Override
      public SortOrder getSortOrder()

      { return this.children.get(0).getSortOrder(); }

      }

      Attachments

        1. PHOENIX-1978.patch
          11 kB
          Rajeshbabu Chintaguntla

        Activity

          People

            rajeshbabu Rajeshbabu Chintaguntla
            alexdl alex kamil
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: