You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "chenhao-db (via GitHub)" <gi...@apache.org> on 2024/03/18 23:58:27 UTC

[PR] [SPARK-47451][SQL] Support to_json(variant). [spark]

chenhao-db opened a new pull request, #45575:
URL: https://github.com/apache/spark/pull/45575

   ### What changes were proposed in this pull request?
   
   This PR adds the functionality to format a variant value as a JSON string. It is exposed in the `to_json` expression by allowing the variant type (or a nested type containing the variant type) as its input.
   
   ### How was this patch tested?
   
   Unit tests that validate the `to_json` result. The input includes both `parse_json` results and manually constructed bytes.
   Negative cases with malformed inputs are also covered.
   
   Some tests disabled in https://github.com/apache/spark/pull/45479 are re-enabled.
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47451][SQL] Support to_json(variant). [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45575:
URL: https://github.com/apache/spark/pull/45575#discussion_r1534017245


##########
common/variant/src/main/java/org/apache/spark/variant/MalformedVariantException.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.variant;
+
+import scala.collection.immutable.Map$;
+
+import org.apache.spark.QueryContext;
+import org.apache.spark.SparkRuntimeException;
+
+public class MalformedVariantException extends SparkRuntimeException {

Review Comment:
   Unless we need to catch this certain exception to do something, it's recommended to simply use `SparkException` or `SparkRuntimeException` (if unchecked exception is needed) with error classes. 



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47451][SQL] Support to_json(variant). [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45575:
URL: https://github.com/apache/spark/pull/45575#discussion_r1534018755


##########
common/variant/src/main/java/org/apache/spark/variant/VariantConstructorSizeLimitException.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.variant;
+
+import scala.collection.immutable.Map$;
+
+import org.apache.spark.QueryContext;
+import org.apache.spark.SparkRuntimeException;
+
+/**
+ * An exception indicating that an external caller tried to call the Variant constructor with value
+ * or metadata exceeding the 16MiB size limit. We will never construct a Variant this large, so it
+ * should only be possible to encounter this exception when reading a Variant produced by another
+ * tool.
+ */
+public class VariantConstructorSizeLimitException extends SparkRuntimeException {

Review Comment:
   ditto



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47451][SQL] Support to_json(variant). [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #45575:
URL: https://github.com/apache/spark/pull/45575#issuecomment-2017343492

   thanks, merging to master!


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47451][SQL] Support to_json(variant). [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45575:
URL: https://github.com/apache/spark/pull/45575#discussion_r1534020418


##########
common/variant/src/main/java/org/apache/spark/variant/MalformedVariantException.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.variant;
+
+import scala.collection.immutable.Map$;
+
+import org.apache.spark.QueryContext;
+import org.apache.spark.SparkRuntimeException;
+
+public class MalformedVariantException extends SparkRuntimeException {

Review Comment:
   end users should distinguish errors by error class, not different java exception class.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47451][SQL] Support to_json(variant). [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #45575:
URL: https://github.com/apache/spark/pull/45575#issuecomment-2016999795

   I made a PR. This PR is blocked by the following.
   - #45685


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47451][SQL] Support to_json(variant). [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45575:
URL: https://github.com/apache/spark/pull/45575#discussion_r1537123309


##########
common/variant/src/main/java/org/apache/spark/types/variant/VariantUtil.java:
##########
@@ -141,4 +148,295 @@ public static byte arrayHeader(boolean largeSize, int offsetSize) {
     return (byte) (((largeSize ? 1 : 0) << (BASIC_TYPE_BITS + 2)) |
         ((offsetSize - 1) << BASIC_TYPE_BITS) | ARRAY);
   }
+
+  // An exception indicating that the variant value or metadata doesn't
+  static SparkRuntimeException malformedVariant() {
+    return new SparkRuntimeException("MALFORMED_VARIANT",
+        Map$.MODULE$.<String, String>empty(), null, new QueryContext[]{}, "");
+  }
+
+  // An exception indicating that an external caller tried to call the Variant constructor with
+  // value or metadata exceeding the 16MiB size limit. We will never construct a Variant this large,
+  // so it should only be possible to encounter this exception when reading a Variant produced by
+  // another tool.
+  static SparkRuntimeException variantConstructorSizeLimit() {
+    return new SparkRuntimeException("VARIANT_CONSTRUCTOR_SIZE_LIMIT",
+        Map$.MODULE$.<String, String>empty(), null, new QueryContext[]{}, "");
+  }
+
+  // Check the validity of an array index `pos`. Throw `MALFORMED_VARIANT` if it is out of bound,
+  // meaning that the variant is malformed.
+  static void checkIndex(int pos, int length) {
+    if (pos < 0 || pos >= length) throw malformedVariant();
+  }
+
+  // Read a little-endian signed long value from `bytes[pos, pos + numBytes)`.
+  static long readLong(byte[] bytes, int pos, int numBytes) {
+    checkIndex(pos, bytes.length);
+    checkIndex(pos + numBytes - 1, bytes.length);
+    long result = 0;
+    // All bytes except the most significant byte should be unsign-extended and shifted (so we need
+    // `& 0xFF`). The most significant byte should be sign-extended and is handled after the loop.
+    for (int i = 0; i < numBytes - 1; ++i) {
+      long unsignedByteValue = bytes[pos + i] & 0xFF;
+      result |= unsignedByteValue << (8 * i);
+    }
+    long signedByteValue = bytes[pos + numBytes - 1];
+    result |= signedByteValue << (8 * (numBytes - 1));
+    return result;
+  }
+
+  // Read a little-endian unsigned int value from `bytes[pos, pos + numBytes)`. The value must fit
+  // into a non-negative int (`[0, Integer.MAX_VALUE]`).
+  static int readUnsigned(byte[] bytes, int pos, int numBytes) {
+    checkIndex(pos, bytes.length);
+    checkIndex(pos + numBytes - 1, bytes.length);
+    int result = 0;
+    // Similar to the `readLong` loop, but all bytes should be unsign-extended.
+    for (int i = 0; i < numBytes; ++i) {
+      int unsignedByteValue = bytes[pos + i] & 0xFF;
+      result |= unsignedByteValue << (8 * i);
+    }
+    if (result < 0) throw malformedVariant();
+    return result;
+  }
+
+  // The value type of variant value. It is determined by the header byte but not a 1:1 mapping
+  // (for example, INT1/2/4/8 all maps to `Type.LONG`).
+  public enum Type {
+    OBJECT,
+    ARRAY,
+    NULL,
+    BOOLEAN,
+    LONG,
+    STRING,
+    DOUBLE,
+    DECIMAL,
+  }
+
+  // Get the value type of variant value `value[pos...]`. It is only legal to call `get*` if
+  // `getType` returns this type (for example, it is only legal to call `getLong` if `getType`
+  // returns `Type.Long`).
+  // Throw `MALFORMED_VARIANT` if the variant is malformed.
+  public static Type getType(byte[] value, int pos) {
+    checkIndex(pos, value.length);
+    int basicType = value[pos] & BASIC_TYPE_MASK;
+    int typeInfo = (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK;
+    switch (basicType) {
+      case SHORT_STR:
+        return Type.STRING;
+      case OBJECT:
+        return Type.OBJECT;
+      case ARRAY:
+        return Type.ARRAY;
+      default:
+        switch (typeInfo) {
+          case NULL:
+            return Type.NULL;
+          case TRUE:
+          case FALSE:
+            return Type.BOOLEAN;
+          case INT1:
+          case INT2:
+          case INT4:
+          case INT8:
+            return Type.LONG;
+          case DOUBLE:
+            return Type.DOUBLE;
+          case DECIMAL4:
+          case DECIMAL8:
+          case DECIMAL16:
+            return Type.DECIMAL;
+          case LONG_STR:
+            return Type.STRING;
+          default:
+            throw malformedVariant();
+        }
+    }
+  }
+
+  static IllegalStateException unexpectedType(Type type) {
+    return new IllegalStateException("Expect type to be " + type);

Review Comment:
   We can address this minor issue in the next VARIANT PR.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47451][SQL] Support to_json(variant). [spark]

Posted by "gene-db (via GitHub)" <gi...@apache.org>.
gene-db commented on PR #45575:
URL: https://github.com/apache/spark/pull/45575#issuecomment-2008378710

   @HyukjinKwon @cloud-fan Could you take a look at this? Thanks!


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47451][SQL] Support to_json(variant). [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45575:
URL: https://github.com/apache/spark/pull/45575#discussion_r1537122374


##########
common/variant/src/main/java/org/apache/spark/types/variant/VariantUtil.java:
##########
@@ -141,4 +148,295 @@ public static byte arrayHeader(boolean largeSize, int offsetSize) {
     return (byte) (((largeSize ? 1 : 0) << (BASIC_TYPE_BITS + 2)) |
         ((offsetSize - 1) << BASIC_TYPE_BITS) | ARRAY);
   }
+
+  // An exception indicating that the variant value or metadata doesn't
+  static SparkRuntimeException malformedVariant() {
+    return new SparkRuntimeException("MALFORMED_VARIANT",
+        Map$.MODULE$.<String, String>empty(), null, new QueryContext[]{}, "");
+  }
+
+  // An exception indicating that an external caller tried to call the Variant constructor with
+  // value or metadata exceeding the 16MiB size limit. We will never construct a Variant this large,
+  // so it should only be possible to encounter this exception when reading a Variant produced by
+  // another tool.
+  static SparkRuntimeException variantConstructorSizeLimit() {
+    return new SparkRuntimeException("VARIANT_CONSTRUCTOR_SIZE_LIMIT",
+        Map$.MODULE$.<String, String>empty(), null, new QueryContext[]{}, "");
+  }
+
+  // Check the validity of an array index `pos`. Throw `MALFORMED_VARIANT` if it is out of bound,
+  // meaning that the variant is malformed.
+  static void checkIndex(int pos, int length) {
+    if (pos < 0 || pos >= length) throw malformedVariant();
+  }
+
+  // Read a little-endian signed long value from `bytes[pos, pos + numBytes)`.
+  static long readLong(byte[] bytes, int pos, int numBytes) {
+    checkIndex(pos, bytes.length);
+    checkIndex(pos + numBytes - 1, bytes.length);
+    long result = 0;
+    // All bytes except the most significant byte should be unsign-extended and shifted (so we need
+    // `& 0xFF`). The most significant byte should be sign-extended and is handled after the loop.
+    for (int i = 0; i < numBytes - 1; ++i) {
+      long unsignedByteValue = bytes[pos + i] & 0xFF;
+      result |= unsignedByteValue << (8 * i);
+    }
+    long signedByteValue = bytes[pos + numBytes - 1];
+    result |= signedByteValue << (8 * (numBytes - 1));
+    return result;
+  }
+
+  // Read a little-endian unsigned int value from `bytes[pos, pos + numBytes)`. The value must fit
+  // into a non-negative int (`[0, Integer.MAX_VALUE]`).
+  static int readUnsigned(byte[] bytes, int pos, int numBytes) {
+    checkIndex(pos, bytes.length);
+    checkIndex(pos + numBytes - 1, bytes.length);
+    int result = 0;
+    // Similar to the `readLong` loop, but all bytes should be unsign-extended.
+    for (int i = 0; i < numBytes; ++i) {
+      int unsignedByteValue = bytes[pos + i] & 0xFF;
+      result |= unsignedByteValue << (8 * i);
+    }
+    if (result < 0) throw malformedVariant();
+    return result;
+  }
+
+  // The value type of variant value. It is determined by the header byte but not a 1:1 mapping
+  // (for example, INT1/2/4/8 all maps to `Type.LONG`).
+  public enum Type {
+    OBJECT,
+    ARRAY,
+    NULL,
+    BOOLEAN,
+    LONG,
+    STRING,
+    DOUBLE,
+    DECIMAL,
+  }
+
+  // Get the value type of variant value `value[pos...]`. It is only legal to call `get*` if
+  // `getType` returns this type (for example, it is only legal to call `getLong` if `getType`
+  // returns `Type.Long`).
+  // Throw `MALFORMED_VARIANT` if the variant is malformed.
+  public static Type getType(byte[] value, int pos) {
+    checkIndex(pos, value.length);
+    int basicType = value[pos] & BASIC_TYPE_MASK;
+    int typeInfo = (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK;
+    switch (basicType) {
+      case SHORT_STR:
+        return Type.STRING;
+      case OBJECT:
+        return Type.OBJECT;
+      case ARRAY:
+        return Type.ARRAY;
+      default:
+        switch (typeInfo) {
+          case NULL:
+            return Type.NULL;
+          case TRUE:
+          case FALSE:
+            return Type.BOOLEAN;
+          case INT1:
+          case INT2:
+          case INT4:
+          case INT8:
+            return Type.LONG;
+          case DOUBLE:
+            return Type.DOUBLE;
+          case DECIMAL4:
+          case DECIMAL8:
+          case DECIMAL16:
+            return Type.DECIMAL;
+          case LONG_STR:
+            return Type.STRING;
+          default:
+            throw malformedVariant();
+        }
+    }
+  }
+
+  static IllegalStateException unexpectedType(Type type) {
+    return new IllegalStateException("Expect type to be " + type);

Review Comment:
   if this should not happen, we should throw `SparkException.internalError`



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47451][SQL] Support to_json(variant). [spark]

Posted by "chenhao-db (via GitHub)" <gi...@apache.org>.
chenhao-db commented on code in PR #45575:
URL: https://github.com/apache/spark/pull/45575#discussion_r1534264815


##########
common/variant/src/main/java/org/apache/spark/variant/MalformedVariantException.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.variant;
+
+import scala.collection.immutable.Map$;
+
+import org.apache.spark.QueryContext;
+import org.apache.spark.SparkRuntimeException;
+
+public class MalformedVariantException extends SparkRuntimeException {

Review Comment:
   Makes sense. I don't think we need to catch this certain exception.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47451][SQL] Support to_json(variant). [spark]

Posted by "chenhao-db (via GitHub)" <gi...@apache.org>.
chenhao-db commented on PR #45575:
URL: https://github.com/apache/spark/pull/45575#issuecomment-2017214483

   I have also performed renaming in this PR. It should be good to merge now.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47451][SQL] Support to_json(variant). [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan closed pull request #45575: [SPARK-47451][SQL] Support to_json(variant).
URL: https://github.com/apache/spark/pull/45575


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47451][SQL] Support to_json(variant). [spark]

Posted by "gene-db (via GitHub)" <gi...@apache.org>.
gene-db commented on code in PR #45575:
URL: https://github.com/apache/spark/pull/45575#discussion_r1530649037


##########
common/variant/src/main/java/org/apache/spark/variant/MalformedVariantException.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.variant;
+
+import scala.collection.immutable.Map$;
+
+import org.apache.spark.QueryContext;
+import org.apache.spark.SparkRuntimeException;
+
+public class MalformedVariantException extends SparkRuntimeException {
+  public MalformedVariantException() {
+    super("MALFORMED_VARIANT", Map$.MODULE$.<String, String>empty(), null, new QueryContext[]{}, "");

Review Comment:
   I think the linter is complaining about this long line.
   
   ```
   Error:  src/main/java/org/apache/spark/variant/MalformedVariantException.java:[27] (sizes) LineLength: Line is longer than 100 characters (found 101).
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org