You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "Chun Chang (JIRA)" <ji...@apache.org> on 2015/04/29 02:48:07 UTC

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

     [ https://issues.apache.org/jira/browse/DRILL-1679?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Chun Chang closed DRILL-1679.
-----------------------------
    Assignee: Chun Chang

verified fix. added a new test case:
Functional/Passing/flatten_operators/general/jira1679.q

{code}
0: jdbc:drill:schema=dfs.drillTestDirAdvanced> with tmp as (select 1 as fixedcolumn, flatten(columns) as newcol from dfs.`/drill/testdata/flatten_operators/jira1679/b.csv`) select regexp_replace(cast(tmp.newcol as VARCHAR(4)), 'a', 'b') from tmp;
+------------+
|   EXPR$0   |
+------------+
| bbcd       |
| bcdb       |
| cdbb       |
+------------+
{code}

> 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
>            Assignee: Chun Chang
>             Fix For: 0.7.0
>
>
> 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)