Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-17160

GetExternalRowField does not properly escape field names, causing generated code not to compile

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • 2.0.0
    • 2.0.1, 2.1.0
    • SQL
    • None

    Description

      The following end-to-end test uncovered a bug in GetExternalRowField:

      import org.apache.spark.sql.functions._
      import org.apache.spark.sql.catalyst.encoders._
      
      spark.sql("set spark.sql.codegen.fallback=false")
      
      val df = Seq(("100-200", "1", "300")).toDF("a", "b", "c")
      val df2 = df.select(regexp_replace($"a", "(\\d+)", "num"))
      df2.mapPartitions(x => x)(RowEncoder(df2.schema)).collect()
      

      This causes

      java.lang.Exception: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 55, Column 64: Invalid escape sequence
      

      The generated code is

      /* 001 */ public Object generate(Object[] references) {
      /* 002 */   return new GeneratedIterator(references);
      /* 003 */ }
      /* 004 */
      /* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
      /* 006 */   private Object[] references;
      /* 007 */   private scala.collection.Iterator inputadapter_input;
      /* 008 */   private java.lang.String serializefromobject_errMsg;
      /* 009 */   private java.lang.String serializefromobject_errMsg1;
      /* 010 */   private UnsafeRow serializefromobject_result;
      /* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
      /* 012 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
      /* 013 */
      /* 014 */   public GeneratedIterator(Object[] references) {
      /* 015 */     this.references = references;
      /* 016 */   }
      /* 017 */
      /* 018 */   public void init(int index, scala.collection.Iterator inputs[]) {
      /* 019 */     partitionIndex = index;
      /* 020 */     inputadapter_input = inputs[0];
      /* 021 */     this.serializefromobject_errMsg = (java.lang.String) references[0];
      /* 022 */     this.serializefromobject_errMsg1 = (java.lang.String) references[1];
      /* 023 */     serializefromobject_result = new UnsafeRow(1);
      /* 024 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32);
      /* 025 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
      /* 026 */   }
      /* 027 */
      /* 028 */   protected void processNext() throws java.io.IOException {
      /* 029 */     while (inputadapter_input.hasNext()) {
      /* 030 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
      /* 031 */       org.apache.spark.sql.Row inputadapter_value = (org.apache.spark.sql.Row)inputadapter_row.get(0, null);
      /* 032 */
      /* 033 */       if (false) {
      /* 034 */         throw new RuntimeException(serializefromobject_errMsg);
      /* 035 */       }
      /* 036 */
      /* 037 */       boolean serializefromobject_isNull1 = false || false;
      /* 038 */       final boolean serializefromobject_value1 = serializefromobject_isNull1 ? false : inputadapter_value.isNullAt(0);
      /* 039 */       boolean serializefromobject_isNull = false;
      /* 040 */       UTF8String serializefromobject_value = null;
      /* 041 */       if (!serializefromobject_isNull1 && serializefromobject_value1) {
      /* 042 */         final UTF8String serializefromobject_value5 = null;
      /* 043 */         serializefromobject_isNull = true;
      /* 044 */         serializefromobject_value = serializefromobject_value5;
      /* 045 */       } else {
      /* 046 */         if (false) {
      /* 047 */           throw new RuntimeException(serializefromobject_errMsg1);
      /* 048 */         }
      /* 049 */
      /* 050 */         if (false) {
      /* 051 */           throw new RuntimeException("The input external row cannot be null.");
      /* 052 */         }
      /* 053 */
      /* 054 */         if (inputadapter_value.isNullAt(0)) {
      /* 055 */           throw new RuntimeException("The 0th field 'regexp_replace(a, (\d+), num)' of input row " +
      /* 056 */             "cannot be null.");
      /* 057 */         }
      /* 058 */
      /* 059 */         final Object serializefromobject_value8 = inputadapter_value.get(0);
      /* 060 */         java.lang.String serializefromobject_value7 = null;
      /* 061 */         if (!false) {
      /* 062 */           if (serializefromobject_value8 instanceof java.lang.String) {
      /* 063 */             serializefromobject_value7 = (java.lang.String) serializefromobject_value8;
      /* 064 */           } else {
      /* 065 */             throw new RuntimeException(serializefromobject_value8.getClass().getName() + " is not a valid " +
      /* 066 */               "external type for schema of string");
      /* 067 */           }
      /* 068 */         }
      /* 069 */         boolean serializefromobject_isNull6 = false;
      /* 070 */         final UTF8String serializefromobject_value6 = serializefromobject_isNull6 ? null : org.apache.spark.unsafe.types.UTF8String.fromString(serializefromobject_value7);
      /* 071 */         serializefromobject_isNull6 = serializefromobject_value6 == null;
      /* 072 */         serializefromobject_isNull = serializefromobject_isNull6;
      /* 073 */         serializefromobject_value = serializefromobject_value6;
      /* 074 */       }
      /* 075 */       serializefromobject_holder.reset();
      /* 076 */
      /* 077 */       serializefromobject_rowWriter.zeroOutNullBytes();
      /* 078 */
      /* 079 */       if (serializefromobject_isNull) {
      /* 080 */         serializefromobject_rowWriter.setNullAt(0);
      /* 081 */       } else {
      /* 082 */         serializefromobject_rowWriter.write(0, serializefromobject_value);
      /* 083 */       }
      /* 084 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
      /* 085 */       append(serializefromobject_result);
      /* 086 */       if (shouldStop()) return;
      /* 087 */     }
      /* 088 */   }
      /* 089 */ }
      

      Here, the problem is that the auto-generated field name contains special characters (including backslashes) and those aren't escaped when being interpolated into the generated code, causing the invalid string literal

      "The 0th field 'regexp_replace(a, (\d+), num)' of input row "
      

      to appear in the generated code.

      We need to update GetExternalRowField to escape field names and also need to audit other expressions to make sure that we're not making the same mistake there.

      Attachments

        Activity

          People

            joshrosen Josh Rosen
            joshrosen Josh Rosen
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: