fqaiser94 commented on pull request #27066:
URL: https://github.com/apache/spark/pull/27066#issuecomment-652443144


   Hey @cloud-fan , I've been attempting to make this idea work for a couple of 
weeks now without success. 
   
   I made some small changes and am able to get the Scala code to compile now. 
   However, I can't get the generated java code to compile. I get the following 
error currently: 
   ```
   [info]   java.util.concurrent.ExecutionException: 
org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 40, 
Column 13: failed to compile: org.codehaus.commons.compiler.CompileException: 
File 'generated.java', Line 40, Column 13: Redefinition of local variable 
"rdd_value_0"
   ```
   Here is an example of the generated java code for the test `withField should 
add field to struct`: 
   ```
   /* 001 */ public Object generate(Object[] references) {
   /* 002 */   return new GeneratedIteratorForCodegenStage1(references);
   /* 003 */ }
   /* 004 */
   /* 005 */ // codegenStageId=1
   /* 006 */ final class GeneratedIteratorForCodegenStage1 extends 
org.apache.spark.sql.execution.BufferedRowIterator {
   /* 007 */   private Object[] references;
   /* 008 */   private scala.collection.Iterator[] inputs;
   /* 009 */   private scala.collection.Iterator rdd_input_0;
   /* 010 */   private 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] 
rdd_mutableStateArray_0 = new 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4];
   /* 011 */
   /* 012 */   public GeneratedIteratorForCodegenStage1(Object[] references) {
   /* 013 */     this.references = references;
   /* 014 */   }
   /* 015 */
   /* 016 */   public void init(int index, scala.collection.Iterator[] inputs) {
   /* 017 */     partitionIndex = index;
   /* 018 */     this.inputs = inputs;
   /* 019 */     rdd_input_0 = inputs[0];
   /* 020 */     rdd_mutableStateArray_0[0] = new 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 32);
   /* 021 */     rdd_mutableStateArray_0[1] = new 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(rdd_mutableStateArray_0[0],
 3);
   /* 022 */     rdd_mutableStateArray_0[2] = new 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 32);
   /* 023 */     rdd_mutableStateArray_0[3] = new 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(rdd_mutableStateArray_0[2],
 4);
   /* 024 */
   /* 025 */   }
   /* 026 */
   /* 027 */   protected void processNext() throws java.io.IOException {
   /* 028 */     while ( rdd_input_0.hasNext()) {
   /* 029 */       InternalRow rdd_row_0 = (InternalRow) rdd_input_0.next();
   /* 030 */       ((org.apache.spark.sql.execution.metric.SQLMetric) 
references[0] /* numOutputRows */).add(1);
   /* 031 */       InternalRow rdd_value_0 = rdd_row_0.getStruct(0, 3);
   /* 032 */       boolean project_isNull_0 = false;
   /* 033 */       InternalRow project_value_0 = null;
   /* 034 */       if (!false && false) {
   /* 035 */         project_isNull_0 = true;
   /* 036 */         project_value_0 = ((InternalRow)null);
   /* 037 */       } else {
   /* 038 */         Object[] project_values_0 = new Object[4];
   /* 039 */
   /* 040 */         InternalRow rdd_value_0 = rdd_row_0.getStruct(0, 3);
   /* 041 */         int project_value_5 = -1;
   /* 042 */
   /* 043 */         project_value_5 = rdd_value_0.getInt(0);
   /* 044 */         if (false) {
   /* 045 */           project_values_0[0] = null;
   /* 046 */         } else {
   /* 047 */           project_values_0[0] = project_value_5;
   /* 048 */         }
   /* 049 */
   /* 050 */         InternalRow rdd_value_0 = rdd_row_0.getStruct(0, 3);
   /* 051 */         boolean project_isNull_7 = false;
   /* 052 */         int project_value_7 = -1;
   /* 053 */
   /* 054 */         if (rdd_value_0.isNullAt(1)) {
   /* 055 */           project_isNull_7 = true;
   /* 056 */         } else {
   /* 057 */           project_value_7 = rdd_value_0.getInt(1);
   /* 058 */         }
   /* 059 */         if (project_isNull_7) {
   /* 060 */           project_values_0[1] = null;
   /* 061 */         } else {
   /* 062 */           project_values_0[1] = project_value_7;
   /* 063 */         }
   /* 064 */
   /* 065 */         InternalRow rdd_value_0 = rdd_row_0.getStruct(0, 3);
   /* 066 */         int project_value_9 = -1;
   /* 067 */
   /* 068 */         project_value_9 = rdd_value_0.getInt(2);
   /* 069 */         if (false) {
   /* 070 */           project_values_0[2] = null;
   /* 071 */         } else {
   /* 072 */           project_values_0[2] = project_value_9;
   /* 073 */         }
   /* 074 */
   /* 075 */         if (false) {
   /* 076 */           project_values_0[3] = null;
   /* 077 */         } else {
   /* 078 */           project_values_0[3] = 4;
   /* 079 */         }
   /* 080 */
   /* 081 */         final InternalRow project_value_4 = new 
org.apache.spark.sql.catalyst.expressions.GenericInternalRow(project_values_0);
   /* 082 */         project_values_0 = null;
   /* 083 */         project_isNull_0 = false;
   /* 084 */         project_value_0 = project_value_4;
   /* 085 */       }
   /* 086 */       rdd_mutableStateArray_0[2].reset();
   /* 087 */
   /* 088 */       rdd_mutableStateArray_0[2].zeroOutNullBytes();
   /* 089 */
   /* 090 */       final InternalRow project_tmpInput_0 = project_value_0;
   /* 091 */       if (project_tmpInput_0 instanceof UnsafeRow) {
   /* 092 */         rdd_mutableStateArray_0[2].write(0, (UnsafeRow) 
project_tmpInput_0);
   /* 093 */       } else {
   /* 094 */         // Remember the current cursor so that we can calculate 
how many bytes are
   /* 095 */         // written later.
   /* 096 */         final int project_previousCursor_0 = 
rdd_mutableStateArray_0[2].cursor();
   /* 097 */
   /* 098 */         rdd_mutableStateArray_0[3].resetRowWriter();
   /* 099 */
   /* 100 */         rdd_mutableStateArray_0[3].write(0, 
(project_tmpInput_0.getInt(0)));
   /* 101 */
   /* 102 */         if ((project_tmpInput_0.isNullAt(1))) {
   /* 103 */           rdd_mutableStateArray_0[3].setNullAt(1);
   /* 104 */         } else {
   /* 105 */           rdd_mutableStateArray_0[3].write(1, 
(project_tmpInput_0.getInt(1)));
   /* 106 */         }
   /* 107 */
   /* 108 */         rdd_mutableStateArray_0[3].write(2, 
(project_tmpInput_0.getInt(2)));
   /* 109 */
   /* 110 */         rdd_mutableStateArray_0[3].write(3, 
(project_tmpInput_0.getInt(3)));
   /* 111 */
   /* 112 */         
rdd_mutableStateArray_0[2].setOffsetAndSizeFromPreviousCursor(0, 
project_previousCursor_0);
   /* 113 */       }
   /* 114 */       append((rdd_mutableStateArray_0[2].getRow()));
   /* 115 */       if (shouldStop()) return;
   /* 116 */     }
   /* 117 */   }
   /* 118 */
   /* 119 */ }
   
   ```
   The error makes sense. You can see this line of code is repeated multiple 
times on lines 31, 40, 50, and 65: 
   ```
   InternalRow rdd_value_0 = rdd_row_0.getStruct(0, 3);
   ```
   It seems that for each `GetStructField` call on the original struct, the 
generated java code is extracting the struct from the row again. 
   I can't figure out how this issue is avoided normally. 
   Do you have any idea what's happening here? 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to