-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-7486] Classify schema exceptions when converting from avro to spark row representation #10778
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
5916630
63c8182
a5ccaf5
ce3b208
5808ef2
d12ec39
6369eb5
42d6607
90a3006
521ae79
0e2e1d8
204bffd
26afcf3
5138020
f7e2e1b
ea630b5
1770608
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,25 +18,25 @@ | |
|
|
||
| package org.apache.hudi | ||
|
|
||
| import org.apache.avro.Schema | ||
| import org.apache.avro.generic.GenericRecord | ||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hudi.HoodieConversionUtils.toScalaOption | ||
| import org.apache.hudi.avro.{AvroSchemaUtils, HoodieAvroUtils} | ||
| import org.apache.hudi.client.utils.SparkRowSerDe | ||
| import org.apache.hudi.common.model.HoodieRecord | ||
| import org.apache.hudi.hadoop.fs.CachingPath | ||
|
|
||
| import org.apache.avro.Schema | ||
| import org.apache.avro.generic.GenericRecord | ||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hudi.util.ExceptionWrappingIterator | ||
| import org.apache.spark.SPARK_VERSION | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.DataFrame | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.util.DateTimeUtils.getTimeZone | ||
| import org.apache.spark.sql.execution.SQLConfInjectingRDD | ||
| import org.apache.spark.sql.execution.datasources.SparkParsePartitionUtil | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.types.{StringType, StructField, StructType} | ||
| import org.apache.spark.sql.{DataFrame, HoodieUnsafeUtils} | ||
| import org.apache.spark.unsafe.types.UTF8String | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
|
|
@@ -131,6 +131,16 @@ object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport wi | |
| def injectSQLConf[T: ClassTag](rdd: RDD[T], conf: SQLConf): RDD[T] = | ||
| new SQLConfInjectingRDD(rdd, conf) | ||
|
|
||
| def maybeWrapDataFrameWithException(df: DataFrame, exceptionClass: String, msg: String, shouldWrap: Boolean): DataFrame = { | ||
| if (shouldWrap) { | ||
| HoodieUnsafeUtils.createDataFrameFromRDD(df.sparkSession, injectSQLConf(df.queryExecution.toRdd.mapPartitions { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is SQLConf injection necessary here?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. #6352 alexey says that "Yes, it will propagate to all RDDs in the execution chain (up to a shuffling point)". So I guess the question is if there is a shuffling point?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. My doubt is if wrapping the df with exception needs the SQL conf injection. So you're saying this needs SQL conf injection so the configs can still be propagated to the executors.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's keep an eye on the DAG and Spark execution when this is rolled out. I think this is OK for now. |
||
| rows => new ExceptionWrappingIterator[InternalRow](rows, exceptionClass, msg) | ||
| }, SQLConf.get), df.schema) | ||
| } else { | ||
| df | ||
| } | ||
| } | ||
|
|
||
| def safeCreateRDD(df: DataFrame, structName: String, recordNamespace: String, reconcileToLatestSchema: Boolean, | ||
| latestTableSchema: org.apache.hudi.common.util.Option[Schema] = org.apache.hudi.common.util.Option.empty()): | ||
| Tuple2[RDD[GenericRecord], RDD[String]] = { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,44 @@ | ||
| /* | ||
| * 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.hudi.util | ||
|
|
||
| import org.apache.hudi.common.util.ReflectionUtils | ||
|
|
||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Docs here? |
||
| /** | ||
| * Used to catch exceptions from an iterator | ||
| * @param in iterator to catch exceptions from | ||
| * @param exceptionClass name of exception class to throw when an exception is thrown during iteration | ||
| * @param msg message the thrown exception should have | ||
| */ | ||
| class ExceptionWrappingIterator[T](val in: Iterator[T], val exceptionClass: String, val msg: String) extends Iterator[T] { | ||
| override def hasNext: Boolean = try in.hasNext | ||
| catch { | ||
| case e: Throwable => throw createException(e) | ||
| } | ||
|
|
||
| override def next: T = try in.next | ||
| catch { | ||
| case e: Throwable => throw createException(e) | ||
| } | ||
|
|
||
| private def createException(e: Throwable): Throwable = { | ||
| ReflectionUtils.loadClass(exceptionClass, Array(classOf[String], classOf[Throwable]).asInstanceOf[Array[Class[_]]], msg, e).asInstanceOf[Throwable] | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,31 @@ | ||
| /* | ||
| * 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.hudi.exception; | ||
|
|
||
| /** | ||
| * Thrown when we detect in Hudi code that a record schema | ||
| * violates Avro rules. This can happen even when using Spark | ||
| * because we use Avro schema internally | ||
| */ | ||
| public class HoodieAvroSchemaException extends SchemaCompatibilityException { | ||
| public HoodieAvroSchemaException(String message) { | ||
| super(message); | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,32 @@ | ||
| /* | ||
| * 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.hudi.exception; | ||
|
|
||
| /** | ||
| * Exception thrown during HoodieRecord construction for any failure | ||
| * that is not a KeyGeneration failure. An example of a failure would be if the | ||
| * record is malformed. | ||
| */ | ||
| public class HoodieRecordCreationException extends HoodieException { | ||
|
|
||
| public HoodieRecordCreationException(String message, Throwable t) { | ||
| super(message, t); | ||
| } | ||
| } |
Uh oh!
There was an error while loading. Please reload this page.