Skip to content
Closed
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -210,10 +210,14 @@ private[hive] case class HiveGenericUDTF(
}

@transient
protected lazy val inputInspectors = children.map(toInspector)
protected lazy val inputInspectors = {
val inspectors = children.map(toInspector)
val fields = inspectors.indices.map(index => s"_col$index").asJava
ObjectInspectorFactory.getStandardStructObjectInspector(fields, inspectors.asJava)
}

@transient
protected lazy val outputInspector = function.initialize(inputInspectors.toArray)
protected lazy val outputInspector = function.initialize(inputInspectors)

@transient
protected lazy val udtInput = new Array[AnyRef](children.length)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,128 @@
/*
* 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.sql.hive.execution;

import org.apache.hadoop.hive.ql.exec.Description;
import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
import org.apache.hadoop.hive.ql.metadata.HiveException;
import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ReturnObjectInspectorResolver;
import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF;
import org.apache.hadoop.hive.serde2.objectinspector.*;
import org.apache.hadoop.io.IntWritable;

import java.util.ArrayList;
import java.util.List;

/**
* Takes a row of size k of data and splits it into n rows of data. For
* example, if n is 3 then the rest of the arguments are split in order into 3
* rows, each of which has k/3 columns in it (the first emitted row has the
* first k/3, the second has the second, etc). If n does not divide k then the
* remaining columns are padded with NULLs.
*/
@Description(
name = "stack",
value = "_FUNC_(n, cols...) - turns k columns into n rows of size k/n each"
)
public class UDTFStack3 extends GenericUDTF {

@Override
public void close() throws HiveException {
}

private transient List<ObjectInspector> argOIs = new ArrayList<ObjectInspector>();
private transient Object[] forwardObj = null;
private transient ArrayList<ReturnObjectInspectorResolver> returnOIResolvers =
new ArrayList<ReturnObjectInspectorResolver>();
IntWritable numRows = null;
Integer numCols = null;

public StructObjectInspector initialize2(ObjectInspector[] args)
throws UDFArgumentException {
if (args.length < 2) {
throw new UDFArgumentException("STACK() expects at least two arguments.");
}
if (!(args[0] instanceof ConstantObjectInspector)) {
throw new UDFArgumentException(
"The first argument to STACK() must be a constant integer (got " +
args[0].getTypeName() + " instead).");
}
numRows = (IntWritable)
((ConstantObjectInspector)args[0]).getWritableConstantValue();

if (numRows == null || numRows.get() < 1) {
throw new UDFArgumentException(
"STACK() expects its first argument to be >= 1.");
}

// Divide and round up.
numCols = (args.length - 1 + numRows.get() - 1) / numRows.get();

for (int jj = 0; jj < numCols; ++jj) {
returnOIResolvers.add(new ReturnObjectInspectorResolver());
for (int ii = 0; ii < numRows.get(); ++ii) {
int index = ii * numCols + jj + 1;
if (index < args.length &&
!returnOIResolvers.get(jj).update(args[index])) {
throw new UDFArgumentException(
"Argument " + (jj + 1) + "'s type (" +
args[jj + 1].getTypeName() + ") should be equal to argument " +
index + "'s type (" + args[index].getTypeName() + ")");
}
}
}

forwardObj = new Object[numCols];
for (int ii = 0; ii < args.length; ++ii) {
argOIs.add(args[ii]);
}

ArrayList<String> fieldNames = new ArrayList<String>();
ArrayList<ObjectInspector> fieldOIs = new ArrayList<ObjectInspector>();
for (int ii = 0; ii < numCols; ++ii) {
fieldNames.add("col" + ii);
fieldOIs.add(returnOIResolvers.get(ii).get());
}

return ObjectInspectorFactory.getStandardStructObjectInspector(
fieldNames, fieldOIs);
}

@Override
public void process(Object[] args)
throws HiveException, UDFArgumentException {
for (int ii = 0; ii < numRows.get(); ++ii) {
for (int jj = 0; jj < numCols; ++jj) {
int index = ii * numCols + jj + 1;
if (index < args.length) {
forwardObj[jj] =
returnOIResolvers.get(jj)
.convertIfNecessary(args[index], argOIs.get(index));
} else {
forwardObj[ii] = null;
}
}
forward(forwardObj);
}
}

@Override
public String toString() {
return "stack";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -2176,8 +2176,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi
sql(
s"""
|CREATE TEMPORARY FUNCTION udtf_stack2
|AS 'org.apache.spark.sql.hive.execution.UDTFStack2'
|USING JAR '${hiveContext.getHiveFile("SPARK-21101-1.0.jar").toURI}'
|AS 'org.apache.spark.sql.hive.execution.UDTFStack3'
""".stripMargin)
val e = intercept[org.apache.spark.sql.AnalysisException] {
sql("SELECT udtf_stack2(2, 'A', 10, date '2015-01-01', 'B', 20, date '2016-01-01')")
Expand Down Expand Up @@ -2560,6 +2559,21 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi
}
}
}

test("SPARK-32668: HiveGenericUDTF initialize UDTF should use StructObjectInspector method") {
withUserDefinedFunction("udtf_stack" -> true) {
// More detail about org.apache.spark.sql.hive.execution.UDTFStack2 could see SPARK-21101.
sql(
s"""
|CREATE TEMPORARY FUNCTION udtf_stack
|AS 'org.apache.spark.sql.hive.execution.UDTFStack2'
|USING JAR '${hiveContext.getHiveFile("SPARK-21101-1.0.jar").toURI}'
""".stripMargin)
val cnt = sql(
"SELECT udtf_stack(2, 'A', 10, date '2015-01-01', 'B', 20, date '2016-01-01')").count()
assert(cnt === 2)
}
}
}

@SlowHiveTest
Expand Down