You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "Hao Zhu (JIRA)" <ji...@apache.org> on 2014/11/10 23:59:34 UTC

[jira] [Created] (DRILL-1679) regexp_replace can not work well with flatten function

Hao Zhu created DRILL-1679:
------------------------------

             Summary: regexp_replace can not work well with flatten function
                 Key: DRILL-1679
                 URL: https://issues.apache.org/jira/browse/DRILL-1679
             Project: Apache Drill
          Issue Type: Bug
          Components: Functions - Drill
    Affects Versions: 0.7.0
         Environment: 0.7.0 with git.commit.id.abbrev=e2ad5b0.
            Reporter: Hao Zhu


I have 2 sample files(a.csv and b.csv)for Drill:
{code}
$  cat a.csv
aaaa
$ cat b.csv
aaaa,bbbb,cccc
{code}

regexp_replace works fine without flatten function on a.csv:
{code}
0: jdbc:drill:zk=local> with tmp as
. . . . . . . . . . . > (
. . . . . . . . . . . > select columns[0] as newcol from `a.csv`
. . . . . . . . . . . > )
. . . . . . . . . . . > select regexp_replace(cast(tmp.newcol as VARCHAR),'a','b')  from tmp;
+------------+
|   EXPR$0   |
+------------+
| bbbb       |
+------------+
1 row selected (0.096 seconds)
{code}

However using flatten function+regexp_replace failed on b.csv:
{code}
0: jdbc:drill:zk=local> with tmp as
. . . . . . . . . . . > (
. . . . . . . . . . . > select 1 as fixedcolumn,flatten(columns) as newcol from `b.csv`
. . . . . . . . . . . > )
. . . . . . . . . . . > select regexp_replace(cast(tmp.newcol as VARCHAR),'a','b')  from tmp;
Query failed: Failure while running fragment., Line 100, Column 21: "value" is neither a method, a field, nor a member class of "org.apache.drill.exec.expr.holders.RepeatedVarCharHolder" [ 84c72b77-b8f7-4c96-b167-905bdf81cbf7 on 10.250.0.115:31010 ]
  (org.apache.drill.exec.exception.SchemaChangeException) Failure while attempting to load generated class
    org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.setupNewSchema():429
    org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.buildSchema():270
    org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.buildSchema():80
    org.apache.drill.exec.physical.impl.ScreenCreator$ScreenRoot.buildSchema():95
    org.apache.drill.exec.work.fragment.FragmentExecutor.run():103
    org.apache.drill.exec.work.WorkManager$RunnableWrapper.run():249
    .......():0
  Caused By (org.apache.drill.exec.exception.ClassTransformationException) java.util.concurrent.ExecutionException: org.apache.drill.exec.exception.ClassTransformationException: Failure generating transformation classes for value: 
 
package org.apache.drill.exec.test.generated;

import java.util.regex.Matcher;
import io.netty.buffer.DrillBuf;
import org.apache.drill.exec.exception.SchemaChangeException;
import org.apache.drill.exec.expr.holders.BigIntHolder;
import org.apache.drill.exec.expr.holders.BitHolder;
import org.apache.drill.exec.expr.holders.RepeatedVarCharHolder;
import org.apache.drill.exec.expr.holders.VarCharHolder;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.record.RecordBatch;
import org.apache.drill.exec.vector.RepeatedVarCharVector;
import org.apache.drill.exec.vector.ValueHolderHelper;
import org.apache.drill.exec.vector.VarCharVector;

public class ProjectorGen99 {

    DrillBuf work0;
    Matcher work1;
    DrillBuf work2;
    RepeatedVarCharVector vv3;
    BigIntHolder constant8;
    VarCharHolder string10;
    VarCharHolder constant11;
    VarCharHolder string12;
    VarCharHolder constant13;
    VarCharVector vv15;

    public void doSetup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing)
        throws SchemaChangeException
    {
        {
            work0 = (incoming).getContext().getManagedBuffer();
            work2 = (incoming).getContext().getManagedBuffer();
            int[] fieldIds4 = new int[ 1 ] ;
            fieldIds4 [ 0 ] = 2;
            Object tmp5 = (incoming).getValueAccessorById(RepeatedVarCharVector.class, fieldIds4).getValueVector();
            if (tmp5 == null) {
                throw new SchemaChangeException("Failure while loading vector vv3 with id: TypedFieldId [fieldIds=[2], remainder=null].");
            }
            vv3 = ((RepeatedVarCharVector) tmp5);
            BigIntHolder out7 = new BigIntHolder();
            out7 .value = 65536L;
            constant8 = out7;
            /** start SETUP for function castVARCHAR **/ 
            {
                BigIntHolder len = constant8;
                DrillBuf buffer = work2;
                 {}
                work2 = buffer;
            }
            /** end SETUP for function castVARCHAR **/ 
            string10 = ValueHolderHelper.getVarCharHolder((incoming).getContext().getManagedBuffer(), "a");
            constant11 = string10;
            string12 = ValueHolderHelper.getVarCharHolder((incoming).getContext().getManagedBuffer(), "b");
            constant13 = string12;
            /** start SETUP for function regexp_replace **/ 
            {
                VarCharHolder pattern = constant11;
                VarCharHolder replacement = constant13;
                DrillBuf buffer = work0;
                Matcher matcher = work1;
                 
StringFunctions$RegexpReplace_setup: {
    matcher = java.util.regex.Pattern.compile(org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(pattern.start, pattern.end, pattern.buffer)).matcher("");
}
 
                work0 = buffer;
                work1 = matcher;
            }
            /** end SETUP for function regexp_replace **/ 
            int[] fieldIds16 = new int[ 1 ] ;
            fieldIds16 [ 0 ] = 0;
            Object tmp17 = (outgoing).getValueAccessorById(VarCharVector.class, fieldIds16).getValueVector();
            if (tmp17 == null) {
                throw new SchemaChangeException("Failure while loading vector vv15 with id: TypedFieldId [fieldIds=[0], remainder=null].");
            }
            vv15 = ((VarCharVector) tmp17);
        }
    }

    public boolean doEval(int inIndex, int outIndex)
        throws SchemaChangeException
    {
        {
            RepeatedVarCharHolder out6 = new RepeatedVarCharHolder();
            {
                vv3 .getAccessor().get((inIndex), out6);
            }
            //---- start of eval portion of castVARCHAR function. ----//
            VarCharHolder out9 = new VarCharHolder();
            {
                final VarCharHolder out = new VarCharHolder();
                RepeatedVarCharHolder in = out6;
                BigIntHolder len = constant8;
                DrillBuf buffer = work2;
                 
SimpleCastFunctions$CastBooleanVarChar_eval: {
    byte[] outB = in.value == 1 ? org.apache.drill.exec.expr.fn.impl.SimpleCastFunctions.TRUE : org.apache.drill.exec.expr.fn.impl.SimpleCastFunctions.FALSE;

    buffer.setBytes(0, outB);
    out.buffer = buffer;
    out.start = 0;
    out.end = Math.min((int) len.value, outB.length);
}
 
                work2 = buffer;
                out9 = out;
            }
            //---- end of eval portion of castVARCHAR function. ----//
            //---- start of eval portion of regexp_replace function. ----//
            VarCharHolder out14 = new VarCharHolder();
            {
                final VarCharHolder out = new VarCharHolder();
                VarCharHolder input = out9;
                VarCharHolder pattern = constant11;
                VarCharHolder replacement = constant13;
                DrillBuf buffer = work0;
                Matcher matcher = work1;
                 
StringFunctions$RegexpReplace_eval: {
    out.start = 0;

    String i = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(input.start, input.end, input.buffer);
    String r = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(replacement.start, replacement.end, replacement.buffer);
    byte[] bytea = matcher.reset(i).replaceAll(r).getBytes(java.nio.charset.Charset.forName("UTF-8"));

    out.buffer = buffer = buffer.reallocIfNeeded(bytea.length);
    out.buffer.setBytes(out.start, bytea);
    out.end = bytea.length;
}
 
                work0 = buffer;
                work1 = matcher;
                out14 = out;
            }
            //---- end of eval portion of regexp_replace function. ----//
            BitHolder out18 = new BitHolder();
            out18 .value = 1;
            if (!vv15 .getMutator().setSafe((outIndex), out14 .start, out14 .end, out14 .buffer)) {
                out18 .value = 0;
            }
            if (out18 .value == 0) {
                return false;
            }
        }
        {
            return true;
        }
    }

    public void __DRILL_INIT__()
        throws SchemaChangeException
    {
    }

}

    org.apache.drill.exec.compile.CodeCompiler.getImplementationClass():59
    org.apache.drill.exec.ops.FragmentContext.getImplementationClass():207
    org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.setupNewSchema():426
    org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.buildSchema():270
    org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.buildSchema():80
    org.apache.drill.exec.physical.impl.ScreenCreator$ScreenRoot.buildSchema():95
    org.apache.drill.exec.work.fragment.FragmentExecutor.run():103
    org.apache.drill.exec.work.WorkManager$RunnableWrapper.run():249
    .......():0
  Caused By (java.util.concurrent.ExecutionException) org.apache.drill.exec.exception.ClassTransformationException: Failure generating transformation classes for value: 
 
package org.apache.drill.exec.test.generated;

import java.util.regex.Matcher;
import io.netty.buffer.DrillBuf;
import org.apache.drill.exec.exception.SchemaChangeException;
import org.apache.drill.exec.expr.holders.BigIntHolder;
import org.apache.drill.exec.expr.holders.BitHolder;
import org.apache.drill.exec.expr.holders.RepeatedVarCharHolder;
import org.apache.drill.exec.expr.holders.VarCharHolder;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.record.RecordBatch;
import org.apache.drill.exec.vector.RepeatedVarCharVector;
import org.apache.drill.exec.vector.ValueHolderHelper;
import org.apache.drill.exec.vector.VarCharVector;

public class ProjectorGen99 {

    DrillBuf work0;
    Matcher work1;
    DrillBuf work2;
    RepeatedVarCharVector vv3;
    BigIntHolder constant8;
    VarCharHolder string10;
    VarCharHolder constant11;
    VarCharHolder string12;
    VarCharHolder constant13;
    VarCharVector vv15;

    public void doSetup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing)
        throws SError: exception while executing query: Failure while executing query. (state=,code=0)
chemaChangeException
    {
        {
            work0 = (incoming).getContext().getManagedBuffer();
            work2 = (incoming).getContext().getManagedBuffer();
            int[] fieldIds4 = new int[ 1 ] ;
            fieldIds4 [ 0 ] = 2;
            Object tmp5 = (incoming).getValueAccessorById(RepeatedVarCharVector.class, fieldIds4).getValueVector();
            if (tmp5 == null) {
                throw new SchemaChangeException("Failure while loading vector vv3 with id: TypedFieldId [fieldIds=[2], remainder=null].");
            }
            vv3 = ((RepeatedVarCharVector) tmp5);
            BigIntHolder out7 = new BigIntHolder();
            out7 .value = 65536L;
            constant8 = out7;
            /** start SETUP for function castVARCHAR **/ 
            {
                BigIntHolder len = constant8;
                DrillBuf buffer = work2;
                 {}
                work2 = buffer;
            }
            /** end SETUP for function castVARCHAR **/ 
            string10 = ValueHolderHelper.getVarCharHolder((incoming).getContext().getManagedBuffer(), "a");
            constant11 = string10;
            string12 = ValueHolderHelper.getVarCharHolder((incoming).getContext().getManagedBuffer(), "b");
            constant13 = string12;
            /** start SETUP for function regexp_replace **/ 
            {
                VarCharHolder pattern = constant11;
                VarCharHolder replacement = constant13;
                DrillBuf buffer = work0;
                Matcher matcher = work1;
                 
StringFunctions$RegexpReplace_setup: {
    matcher = java.util.regex.Pattern.compile(org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(pattern.start, pattern.end, pattern.buffer)).matcher("");
}
 
                work0 = buffer;
                work1 = matcher;
            }
            /** end SETUP for function regexp_replace **/ 
            int[] fieldIds16 = new int[ 1 ] ;
            fieldIds16 [ 0 ] = 0;
            Object tmp17 = (outgoing).getValueAccessorById(VarCharVector.class, fieldIds16).getValueVector();
            if (tmp17 == null) {
                throw new SchemaChangeException("Failure while loading vector vv15 with id: TypedFieldId [fieldIds=[0], remainder=null].");
            }
            vv15 = ((VarCharVector) tmp17);
        }
    }

    public boolean doEval(int inIndex, int outIndex)
        throws SchemaChangeException
    {
        {
            RepeatedVarCharHolder out6 = new RepeatedVarCharHolder();
            {
                vv3 .getAccessor().get((inIndex), out6);
            }
            //---- start of eval portion of castVARCHAR function. ----//
            VarCharHolder out9 = new VarCharHolder();
            {
                final VarCharHolder out = new VarCharHolder();
                RepeatedVarCharHolder in = out6;
                BigIntHolder len = constant8;
                DrillBuf buffer = work2;
                 
SimpleCastFunctions$CastBooleanVarChar_eval: {
    byte[] outB = in.value == 1 ? org.apache.drill.exec.expr.fn.impl.SimpleCastFunctions.TRUE : org.apache.drill.exec.expr.fn.impl.SimpleCastFunctions.FALSE;

    buffer.setBytes(0, outB);
    out.buffer = buffer;
    out.start = 0;
    out.end = Math.min((int) len.value, outB.length);
}
 
                work2 = buffer;
                out9 = out;
            }
            //---- end of eval portion of castVARCHAR function. ----//
            //---- start of eval portion of regexp_replace function. ----//
            VarCharHolder out14 = new VarCharHolder();
            {
                final VarCharHolder out = new VarCharHolder();
                VarCharHolder input = out9;
                VarCharHolder pattern = constant11;
                VarCharHolder replacement = constant13;
                DrillBuf buffer = work0;
                Matcher matcher = work1;
                 
StringFunctions$RegexpReplace_eval: {
    out.start = 0;

    String i = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(input.start, input.end, input.buffer);
    String r = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(replacement.start, replacement.end, replacement.buffer);
    byte[] bytea = matcher.reset(i).replaceAll(r).getBytes(java.nio.charset.Charset.forName("UTF-8"));

    out.buffer = buffer = buffer.reallocIfNeeded(bytea.length);
    out.buffer.setBytes(out.start, bytea);
    out.end = bytea.length;
}
 
                work0 = buffer;
                work1 = matcher;
                out14 = out;
            }
            //---- end of eval portion of regexp_replace function. ----//
            BitHolder out18 = new BitHolder();
            out18 .value = 1;
            if (!vv15 .getMutator().setSafe((outIndex), out14 .start, out14 .end, out14 .buffer)) {
                out18 .value = 0;
            }
            if (out18 .value == 0) {
                return false;
            }
        }
        {
            return true;
        }
    }

    public void __DRILL_INIT__()
        throws SchemaChangeException
    {
    }

}

    com.google.common.util.concurrent.AbstractFuture$Sync.getValue():306
    com.google.common.util.concurrent.AbstractFuture$Sync.get():293
    com.google.common.util.concurrent.AbstractFuture.get():116
    com.google.common.util.concurrent.Uninterruptibles.getUninterruptibly():135
    com.google.common.cache.LocalCache$Segment.getAndRecordStats():2410
    com.google.common.cache.LocalCache$Segment.loadSync():2380
    com.google.common.cache.LocalCache$Segment.lockedGetOrLoad():2342
    com.google.common.cache.LocalCache$Segment.get():2257
    com.google.common.cache.LocalCache.get():4000
    com.google.common.cache.LocalCache.getOrLoad():4004
    com.google.common.cache.LocalCache$LocalLoadingCache.get():4874
    org.apache.drill.exec.compile.CodeCompiler.getImplementationClass():56
    org.apache.drill.exec.ops.FragmentContext.getImplementationClass():207
    org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.setupNewSchema():426
    org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.buildSchema():270
    org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.buildSchema():80
    org.apache.drill.exec.physical.impl.ScreenCreator$ScreenRoot.buildSchema():95
    org.apache.drill.exec.work.fragment.FragmentExecutor.run():103
    org.apache.drill.exec.work.WorkManager$RunnableWrapper.run():249
    .......():0
  Caused By (org.apache.drill.exec.exception.ClassTransformationException) Failure generating transformation classes for value: 
 
package org.apache.drill.exec.test.generated;

import java.util.regex.Matcher;
import io.netty.buffer.DrillBuf;
import org.apache.drill.exec.exception.SchemaChangeException;
import org.apache.drill.exec.expr.holders.BigIntHolder;
import org.apache.drill.exec.expr.holders.BitHolder;
import org.apache.drill.exec.expr.holders.RepeatedVarCharHolder;
import org.apache.drill.exec.expr.holders.VarCharHolder;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.record.RecordBatch;
import org.apache.drill.exec.vector.RepeatedVarCharVector;
import org.apache.drill.exec.vector.ValueHolderHelper;
import org.apache.drill.exec.vector.VarCharVector;

public class ProjectorGen99 {

    DrillBuf work0;
    Matcher work1;
    DrillBuf work2;
    RepeatedVarCharVector vv3;
    BigIntHolder constant8;
    VarCharHolder string10;
    VarCharHolder constant11;
    VarCharHolder string12;
    VarCharHolder constant13;
    VarCharVector vv15;

    public void doSetup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing)
        throws SchemaChangeException
    {
        {
            work0 = (incoming).getContext().getManagedBuffer();
            work2 = (incoming).getContext().getManagedBuffer();
            int[] fieldIds4 = new int[ 1 ] ;
            fieldIds4 [ 0 ] = 2;
            Object tmp5 = (incoming).getValueAccessorById(RepeatedVarCharVector.class, fieldIds4).getValueVector();
            if (tmp5 == null) {
                throw new SchemaChangeException("Failure while loading vector vv3 with id: TypedFieldId [fieldIds=[2], remainder=null].");
            }
            vv3 = ((RepeatedVarCharVector) tmp5);
            BigIntHolder out7 = new BigIntHolder();
            out7 .value = 65536L;
            constant8 = out7;
            /** start SETUP for function castVARCHAR **/ 
            {
                BigIntHolder len = constant8;
                DrillBuf buffer = work2;
                 {}
                work2 = buffer;
            }
            /** end SETUP for function castVARCHAR **/ 
            string10 = ValueHolderHelper.getVarCharHolder((incoming).getContext().getManagedBuffer(), "a");
            constant11 = string10;
            string12 = ValueHolderHelper.getVarCharHolder((incoming).getContext().getManagedBuffer(), "b");
            constant13 = string12;
            /** start SETUP for function regexp_replace **/ 
            {
                VarCharHolder pattern = constant11;
                VarCharHolder replacement = constant13;
                DrillBuf buffer = work0;
                Matcher matcher = work1;
                 
StringFunctions$RegexpReplace_setup: {
    matcher = java.util.regex.Pattern.compile(org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(pattern.start, pattern.end, pattern.buffer)).matcher("");
}
 
                work0 = buffer;
                work1 = matcher;
            }
            /** end SETUP for function regexp_replace **/ 
            int[] fieldIds16 = new int[ 1 ] ;
            fieldIds16 [ 0 ] = 0;
            Object tmp17 = (outgoing).getValueAccessorById(VarCharVector.class, fieldIds16).getValueVector();
            if (tmp17 == null) {
                throw new SchemaChangeException("Failure while loading vector vv15 with id: TypedFieldId [fieldIds=[0], remainder=null].");
            }
            vv15 = ((VarCharVector) tmp17);
        }
    }

    public boolean doEval(int inIndex, int outIndex)
        throws SchemaChangeException
    {
        {
            RepeatedVarCharHolder out6 = new RepeatedVarCharHolder();
            {
                vv3 .getAccessor().get((inIndex), out6);
            }
            //---- start of eval portion of castVARCHAR function. ----//
            VarCharHolder out9 = new VarCharHolder();
            {
                final VarCharHolder out = new VarCharHolder();
                RepeatedVarCharHolder in = out6;
                BigIntHolder len = constant8;
                DrillBuf buffer = work2;
                 
SimpleCastFunctions$CastBooleanVarChar_eval: {
    byte[] outB = in.value == 1 ? org.apache.drill.exec.expr.fn.impl.SimpleCastFunctions.TRUE : org.apache.drill.exec.expr.fn.impl.SimpleCastFunctions.FALSE;

    buffer.setBytes(0, outB);
    out.buffer = buffer;
    out.start = 0;
    out.end = Math.min((int) len.value, outB.length);
}
 
                work2 = buffer;
                out9 = out;
            }
            //---- end of eval portion of castVARCHAR function. ----//
            //---- start of eval portion of regexp_replace function. ----//
            VarCharHolder out14 = new VarCharHolder();
            {
                final VarCharHolder out = new VarCharHolder();
                VarCharHolder input = out9;
                VarCharHolder pattern = constant11;
                VarCharHolder replacement = constant13;
                DrillBuf buffer = work0;
                Matcher matcher = work1;
                 
StringFunctions$RegexpReplace_eval: {
    out.start = 0;

    String i = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(input.start, input.end, input.buffer);
    String r = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(replacement.start, replacement.end, replacement.buffer);
    byte[] bytea = matcher.reset(i).replaceAll(r).getBytes(java.nio.charset.Charset.forName("UTF-8"));

    out.buffer = buffer = buffer.reallocIfNeeded(bytea.length);
    out.buffer.setBytes(out.start, bytea);
    out.end = bytea.length;
}
 
                work0 = buffer;
                work1 = matcher;
                out14 = out;
            }
            //---- end of eval portion of regexp_replace function. ----//
            BitHolder out18 = new BitHolder();
            out18 .value = 1;
            if (!vv15 .getMutator().setSafe((outIndex), out14 .start, out14 .end, out14 .buffer)) {
                out18 .value = 0;
            }
            if (out18 .value == 0) {
                return false;
            }
        }
        {
            return true;
        }
    }

    public void __DRILL_INIT__()
        throws SchemaChangeException
    {
    }

}

    org.apache.drill.exec.compile.ClassTransformer.getImplementationClass():233
    org.apache.drill.exec.compile.CodeCompiler$Loader.load():67
    org.apache.drill.exec.compile.CodeCompiler$Loader.load():63
    com.google.common.cache.LocalCache$LoadingValueReference.loadFuture():3599
    com.google.common.cache.LocalCache$Segment.loadSync():2379
    com.google.common.cache.LocalCache$Segment.lockedGetOrLoad():2342
    com.google.common.cache.LocalCache$Segment.get():2257
    com.google.common.cache.LocalCache.get():4000
    com.google.common.cache.LocalCache.getOrLoad():4004
    com.google.common.cache.LocalCache$LocalLoadingCache.get():4874
    org.apache.drill.exec.compile.CodeCompiler.getImplementationClass():56
    org.apache.drill.exec.ops.FragmentContext.getImplementationClass():207
    org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.setupNewSchema():426
    org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.buildSchema():270
    org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.buildSchema():80
    org.apache.drill.exec.physical.impl.ScreenCreator$ScreenRoot.buildSchema():95
    org.apache.drill.exec.work.fragment.FragmentExecutor.run():103
    org.apache.drill.exec.work.WorkManager$RunnableWrapper.run():249
    .......():0
  Caused By (org.codehaus.commons.compiler.CompileException) Line 100, Column 21: "value" is neither a method, a field, nor a member class of "org.apache.drill.exec.expr.holders.RepeatedVarCharHolder"
    org.codehaus.janino.UnitCompiler.compileError():10056
    org.codehaus.janino.UnitCompiler.reclassifyName():6780
    org.codehaus.janino.UnitCompiler.reclassify():6700
    org.codehaus.janino.UnitCompiler.getConstantValue2():4426
    org.codehaus.janino.UnitCompiler.access$9900():182
    org.codehaus.janino.UnitCompiler$11.visitAmbiguousName():4403
    org.codehaus.janino.Java$AmbiguousName.accept():3142
    org.codehaus.janino.UnitCompiler.getConstantValue():4413
    org.codehaus.janino.UnitCompiler.getConstantValue2():4484
    org.codehaus.janino.UnitCompiler.access$8900():182
    org.codehaus.janino.UnitCompiler$11.visitBinaryOperation():4380
    org.codehaus.janino.Java$BinaryOperation.accept():3772
    org.codehaus.janino.UnitCompiler.getConstantValue():4413
    org.codehaus.janino.UnitCompiler.getConstantValue2():4451
    org.codehaus.janino.UnitCompiler.access$9100():182
    org.codehaus.janino.UnitCompiler$11.visitConditionalExpression():4383
    org.codehaus.janino.Java$ConditionalExpression.accept():3445
    org.codehaus.janino.UnitCompiler.getConstantValue():4413
    org.codehaus.janino.UnitCompiler.compileGetValue():4346
    org.codehaus.janino.UnitCompiler.compile2():1835
    org.codehaus.janino.UnitCompiler.access$2000():182
    org.codehaus.janino.UnitCompiler$4.visitLocalVariableDeclarationStatement():949
    org.codehaus.janino.Java$LocalVariableDeclarationStatement.accept():2512
    org.codehaus.janino.UnitCompiler.compile():962
    org.codehaus.janino.UnitCompiler.compileStatements():1004
    org.codehaus.janino.UnitCompiler.compile2():989
    org.codehaus.janino.UnitCompiler.access$1000():182
    org.codehaus.janino.UnitCompiler$4.visitBlock():939
    org.codehaus.janino.Java$Block.accept():2005
    org.codehaus.janino.UnitCompiler.compile():962
    org.codehaus.janino.UnitCompiler.compile2():1327
    org.codehaus.janino.UnitCompiler.access$900():182
    org.codehaus.janino.UnitCompiler$4.visitLabeledStatement():938
    org.codehaus.janino.Java$LabeledStatement.accept():1966
    org.codehaus.janino.UnitCompiler.compile():962
    org.codehaus.janino.UnitCompiler.compileStatements():1004
    org.codehaus.janino.UnitCompiler.compile2():989
    org.codehaus.janino.UnitCompiler.access$1000():182
    org.codehaus.janino.UnitCompiler$4.visitBlock():939
    org.codehaus.janino.Java$Block.accept():2005
    org.codehaus.janino.UnitCompiler.compile():962
    org.codehaus.janino.UnitCompiler.compileStatements():1004
    org.codehaus.janino.UnitCompiler.compile2():989
    org.codehaus.janino.UnitCompiler.access$1000():182
    org.codehaus.janino.UnitCompiler$4.visitBlock():939
    org.codehaus.janino.Java$Block.accept():2005
    org.codehaus.janino.UnitCompiler.compile():962
    org.codehaus.janino.UnitCompiler.compileStatements():1004
    org.codehaus.janino.UnitCompiler.compile():2284
    org.codehaus.janino.UnitCompiler.compileDeclaredMethods():826
    org.codehaus.janino.UnitCompiler.compileDeclaredMethods():798
    org.codehaus.janino.UnitCompiler.compile2():503
    org.codehaus.janino.UnitCompiler.compile2():389
    org.codehaus.janino.UnitCompiler.access$400():182
    org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration():343
    org.codehaus.janino.Java$PackageMemberClassDeclaration.accept():1136
    org.codehaus.janino.UnitCompiler.compile():350
    org.codehaus.janino.UnitCompiler.compileUnit():318
    org.apache.drill.exec.compile.JaninoClassCompiler.getByteCode():50
    org.apache.drill.exec.compile.AbstractClassCompiler.getClassByteCode():39
    org.apache.drill.exec.compile.QueryClassLoader$ClassCompilerSelector.getClassByteCode():140
    org.apache.drill.exec.compile.QueryClassLoader$ClassCompilerSelector.access$000():109
    org.apache.drill.exec.compile.QueryClassLoader.getClassByteCode():102
    org.apache.drill.exec.compile.ClassTransformer.getImplementationClass():192
    org.apache.drill.exec.compile.CodeCompiler$Loader.load():67
    org.apache.drill.exec.compile.CodeCompiler$Loader.load():63
    com.google.common.cache.LocalCache$LoadingValueReference.loadFuture():3599
    com.google.common.cache.LocalCache$Segment.loadSync():2379
    com.google.common.cache.LocalCache$Segment.lockedGetOrLoad():2342
    com.google.common.cache.LocalCache$Segment.get():2257
    com.google.common.cache.LocalCache.get():4000
    com.google.common.cache.LocalCache.getOrLoad():4004
    com.google.common.cache.LocalCache$LocalLoadingCache.get():4874
    org.apache.drill.exec.compile.CodeCompiler.getImplementationClass():56
    org.apache.drill.exec.ops.FragmentContext.getImplementationClass():207
    org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.setupNewSchema():426
    org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.buildSchema():270
    org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.buildSchema():80
    org.apache.drill.exec.physical.impl.ScreenCreator$ScreenRoot.buildSchema():95
    org.apache.drill.exec.work.fragment.FragmentExecutor.run():103
    org.apache.drill.exec.work.WorkManager$RunnableWrapper.run():249
    .......():0
{code}

The "tmp" with-subquery in above query on b.csv works fine:
{code}
0: jdbc:drill:zk=local> select 1 as fixedcolumn,flatten(columns) as newcol from `b.csv`
. . . . . . . . . . . > ;
+-------------+------------+
| fixedcolumn |   newcol   |
+-------------+------------+
| 1           | aaaa       |
| 1           | bbbb       |
| 1           | cccc       |
+-------------+------------+
3 rows selected (0.088 seconds)
{code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)