You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/10/21 16:13:34 UTC

[GitHub] [spark] sunchao commented on a diff in pull request #38277: [SPARK-40815][SQL] Introduce DelegateSymlinkTextInputFormat to handle empty splits when "spark.hadoopRDD.ignoreEmptySplits" is enabled in order to fix the correctness issue Hive SymlinkTextInputFormat

sunchao commented on code in PR #38277:
URL: https://github.com/apache/spark/pull/38277#discussion_r1001963440


##########
sql/hive/src/main/java/org/apache/hadoop/hive/ql/io/DelegateSymlinkTextInputFormat.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.hadoop.hive.ql.io;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * Delegate for SymlinkTextInputFormat, created to address SPARK-40815.
+ * Fixes an issue where SymlinkTextInputFormat returns empty splits which could result in
+ * the correctness issue when "spark.hadoopRDD.ignoreEmptySplits" is enabled.
+ *
+ * In this class, we update the split start and length to match the target file input thus fixing
+ * the issue.
+ */
+@SuppressWarnings("deprecation")
+public class DelegateSymlinkTextInputFormat extends SymlinkTextInputFormat {
+
+  public static class DelegateSymlinkTextInputSplit extends FileSplit {
+    private final SymlinkTextInputSplit split;
+
+    public DelegateSymlinkTextInputSplit() {
+      super((Path) null, 0, 0, (String[]) null);
+      split = new SymlinkTextInputSplit();
+    }
+
+    public DelegateSymlinkTextInputSplit(Path symlinkPath, SymlinkTextInputSplit split) throws IOException {
+      // It is fine to set start and length to the target file split because
+      // SymlinkTextInputFormat maintains 1-1 mapping between SymlinkTextInputSplit and FileSplit.
+      super(symlinkPath,
+        split.getTargetSplit().getStart(),
+        split.getTargetSplit().getLength(),
+        split.getTargetSplit().getLocations());
+      this.split = split;
+    }
+
+    /**
+     * Returns delegate input split.
+     */
+    private SymlinkTextInputSplit getSplit() {
+      return split;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      super.write(out);

Review Comment:
   hmm won't this call `FileSplit.write` twice?



##########
sql/hive/src/main/java/org/apache/hadoop/hive/ql/io/DelegateSymlinkTextInputFormat.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.hadoop.hive.ql.io;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * Delegate for SymlinkTextInputFormat, created to address SPARK-40815.
+ * Fixes an issue where SymlinkTextInputFormat returns empty splits which could result in
+ * the correctness issue when "spark.hadoopRDD.ignoreEmptySplits" is enabled.
+ *
+ * In this class, we update the split start and length to match the target file input thus fixing
+ * the issue.
+ */
+@SuppressWarnings("deprecation")
+public class DelegateSymlinkTextInputFormat extends SymlinkTextInputFormat {
+
+  public static class DelegateSymlinkTextInputSplit extends FileSplit {
+    private final SymlinkTextInputSplit split;
+
+    public DelegateSymlinkTextInputSplit() {
+      super((Path) null, 0, 0, (String[]) null);
+      split = new SymlinkTextInputSplit();
+    }
+
+    public DelegateSymlinkTextInputSplit(Path symlinkPath, SymlinkTextInputSplit split) throws IOException {
+      // It is fine to set start and length to the target file split because
+      // SymlinkTextInputFormat maintains 1-1 mapping between SymlinkTextInputSplit and FileSplit.
+      super(symlinkPath,
+        split.getTargetSplit().getStart(),
+        split.getTargetSplit().getLength(),
+        split.getTargetSplit().getLocations());
+      this.split = split;
+    }
+
+    /**
+     * Returns delegate input split.
+     */
+    private SymlinkTextInputSplit getSplit() {
+      return split;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      super.write(out);
+      split.write(out);
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      super.readFields(in);

Review Comment:
   ditto



##########
sql/hive/src/main/java/org/apache/hadoop/hive/ql/io/DelegateSymlinkTextInputFormat.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.hadoop.hive.ql.io;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * Delegate for SymlinkTextInputFormat, created to address SPARK-40815.
+ * Fixes an issue where SymlinkTextInputFormat returns empty splits which could result in
+ * the correctness issue when "spark.hadoopRDD.ignoreEmptySplits" is enabled.
+ *
+ * In this class, we update the split start and length to match the target file input thus fixing
+ * the issue.
+ */
+@SuppressWarnings("deprecation")
+public class DelegateSymlinkTextInputFormat extends SymlinkTextInputFormat {
+
+  public static class DelegateSymlinkTextInputSplit extends FileSplit {
+    private final SymlinkTextInputSplit split;
+
+    public DelegateSymlinkTextInputSplit() {
+      super((Path) null, 0, 0, (String[]) null);
+      split = new SymlinkTextInputSplit();
+    }
+
+    public DelegateSymlinkTextInputSplit(Path symlinkPath, SymlinkTextInputSplit split) throws IOException {

Review Comment:
   nit: do we need `symlinkPath`? it can be replaced by `split.getPath()`.



##########
sql/hive/src/main/java/org/apache/hadoop/hive/ql/io/DelegateSymlinkTextInputFormat.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.hadoop.hive.ql.io;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * Delegate for SymlinkTextInputFormat, created to address SPARK-40815.
+ * Fixes an issue where SymlinkTextInputFormat returns empty splits which could result in
+ * the correctness issue when "spark.hadoopRDD.ignoreEmptySplits" is enabled.
+ *
+ * In this class, we update the split start and length to match the target file input thus fixing
+ * the issue.
+ */
+@SuppressWarnings("deprecation")
+public class DelegateSymlinkTextInputFormat extends SymlinkTextInputFormat {
+
+  public static class DelegateSymlinkTextInputSplit extends FileSplit {
+    private final SymlinkTextInputSplit split;
+
+    public DelegateSymlinkTextInputSplit() {

Review Comment:
   why we need this constructor?



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