You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2014/09/02 18:30:34 UTC
svn commit: r1622060 [3/3] - in /hive/trunk:
beeline/src/java/org/apache/hive/beeline/
common/src/java/org/apache/hadoop/hive/common/
common/src/java/org/apache/hadoop/hive/conf/
hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/ hc...
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLastValue.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLastValue.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLastValue.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLastValue.java Tue Sep 2 16:30:33 2014
@@ -37,131 +37,107 @@ import org.apache.hadoop.hive.serde2.obj
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-@WindowFunctionDescription(description = @Description(name = "last_value", value = "_FUNC_(x)"), supportsWindow = true, pivotResult = false, impliesOrder = true)
-public class GenericUDAFLastValue extends AbstractGenericUDAFResolver
-{
- static final Log LOG = LogFactory.getLog(GenericUDAFLastValue.class
- .getName());
-
- @Override
- public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
- throws SemanticException
- {
- if (parameters.length > 2)
- {
- throw new UDFArgumentTypeException(2, "At most 2 arguments expected");
- }
- if ( parameters.length > 1 && !parameters[1].equals(TypeInfoFactory.booleanTypeInfo) )
- {
- throw new UDFArgumentTypeException(1, "second argument must be a boolean expression");
- }
- return createEvaluator();
- }
-
- protected GenericUDAFLastValueEvaluator createEvaluator()
- {
- return new GenericUDAFLastValueEvaluator();
- }
-
- static class LastValueBuffer implements AggregationBuffer
- {
- Object val;
- boolean firstRow;
- boolean skipNulls;
-
- LastValueBuffer()
- {
- init();
- }
-
- void init()
- {
- val = null;
- firstRow = true;
- skipNulls = false;
- }
-
- }
-
- public static class GenericUDAFLastValueEvaluator extends
- GenericUDAFEvaluator
- {
- ObjectInspector inputOI;
- ObjectInspector outputOI;
-
- @Override
- public ObjectInspector init(Mode m, ObjectInspector[] parameters)
- throws HiveException
- {
- super.init(m, parameters);
- if (m != Mode.COMPLETE)
- {
- throw new HiveException(
- "Only COMPLETE mode supported for Rank function");
- }
- inputOI = parameters[0];
- outputOI = ObjectInspectorUtils.getStandardObjectInspector(inputOI,
- ObjectInspectorCopyOption.WRITABLE);
- return outputOI;
- }
-
- @Override
- public AggregationBuffer getNewAggregationBuffer() throws HiveException
- {
- return new LastValueBuffer();
- }
-
- @Override
- public void reset(AggregationBuffer agg) throws HiveException
- {
- ((LastValueBuffer) agg).init();
- }
-
- @Override
- public void iterate(AggregationBuffer agg, Object[] parameters)
- throws HiveException
- {
- LastValueBuffer lb = (LastValueBuffer) agg;
- if (lb.firstRow )
- {
- lb.firstRow = false;
- if ( parameters.length == 2 )
- {
- lb.skipNulls = PrimitiveObjectInspectorUtils.getBoolean(
- parameters[1],
- PrimitiveObjectInspectorFactory.writableBooleanObjectInspector);
- }
- }
-
- Object o = ObjectInspectorUtils.copyToStandardObject(parameters[0],
- inputOI, ObjectInspectorCopyOption.WRITABLE);
+@WindowFunctionDescription(description = @Description(name = "last_value", value = "_FUNC_(x)"),
+ supportsWindow = true, pivotResult = false, impliesOrder = true)
+public class GenericUDAFLastValue extends AbstractGenericUDAFResolver {
+
+ static final Log LOG = LogFactory.getLog(GenericUDAFLastValue.class.getName());
+
+ @Override
+ public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
+ if (parameters.length > 2) {
+ throw new UDFArgumentTypeException(2, "At most 2 arguments expected");
+ }
+ if (parameters.length > 1 && !parameters[1].equals(TypeInfoFactory.booleanTypeInfo)) {
+ throw new UDFArgumentTypeException(1, "second argument must be a boolean expression");
+ }
+ return createEvaluator();
+ }
+
+ protected GenericUDAFLastValueEvaluator createEvaluator() {
+ return new GenericUDAFLastValueEvaluator();
+ }
+
+ static class LastValueBuffer implements AggregationBuffer {
+
+ Object val;
+ boolean firstRow;
+ boolean skipNulls;
+
+ LastValueBuffer() {
+ init();
+ }
+
+ void init() {
+ val = null;
+ firstRow = true;
+ skipNulls = false;
+ }
+
+ }
+
+ public static class GenericUDAFLastValueEvaluator extends GenericUDAFEvaluator {
+
+ ObjectInspector inputOI;
+ ObjectInspector outputOI;
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ super.init(m, parameters);
+ if (m != Mode.COMPLETE) {
+ throw new HiveException("Only COMPLETE mode supported for Rank function");
+ }
+ inputOI = parameters[0];
+ outputOI = ObjectInspectorUtils.getStandardObjectInspector(inputOI,
+ ObjectInspectorCopyOption.WRITABLE);
+ return outputOI;
+ }
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ return new LastValueBuffer();
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ ((LastValueBuffer) agg).init();
+ }
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ LastValueBuffer lb = (LastValueBuffer) agg;
+ if (lb.firstRow) {
+ lb.firstRow = false;
+ if (parameters.length == 2) {
+ lb.skipNulls = PrimitiveObjectInspectorUtils.getBoolean(parameters[1],
+ PrimitiveObjectInspectorFactory.writableBooleanObjectInspector);
+ }
+ }
+
+ Object o = ObjectInspectorUtils.copyToStandardObject(parameters[0], inputOI,
+ ObjectInspectorCopyOption.WRITABLE);
if (!lb.skipNulls || o != null) {
lb.val = o;
}
- }
+ }
- @Override
- public Object terminatePartial(AggregationBuffer agg)
- throws HiveException
- {
- throw new HiveException("terminatePartial not supported");
- }
-
- @Override
- public void merge(AggregationBuffer agg, Object partial)
- throws HiveException
- {
- throw new HiveException("merge not supported");
- }
-
- @Override
- public Object terminate(AggregationBuffer agg) throws HiveException
- {
- LastValueBuffer lb = (LastValueBuffer) agg;
- return lb.val;
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ throw new HiveException("terminatePartial not supported");
+ }
- }
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ throw new HiveException("merge not supported");
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ LastValueBuffer lb = (LastValueBuffer) agg;
+ return lb.val;
+
+ }
@Override
public GenericUDAFEvaluator getWindowingEvaluator(WindowFrameDef wFrmDef) {
@@ -169,12 +145,12 @@ public class GenericUDAFLastValue extend
BoundaryDef end = wFrmDef.getEnd();
return new LastValStreamingFixedWindow(this, start.getAmt(), end.getAmt());
}
- }
+ }
- static class LastValStreamingFixedWindow extends
- GenericUDAFStreamingEvaluator<Object> {
+ static class LastValStreamingFixedWindow extends GenericUDAFStreamingEvaluator<Object> {
class State extends GenericUDAFStreamingEvaluator<Object>.StreamingState {
+
private Object lastValue;
private int lastIdx;
@@ -203,8 +179,8 @@ public class GenericUDAFLastValue extend
}
}
- public LastValStreamingFixedWindow(GenericUDAFEvaluator wrappedEval,
- int numPreceding, int numFollowing) {
+ public LastValStreamingFixedWindow(GenericUDAFEvaluator wrappedEval, int numPreceding,
+ int numFollowing) {
super(wrappedEval, numPreceding, numFollowing);
}
@@ -224,8 +200,7 @@ public class GenericUDAFLastValue extend
}
@Override
- public void iterate(AggregationBuffer agg, Object[] parameters)
- throws HiveException {
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
State s = (State) agg;
LastValueBuffer lb = (LastValueBuffer) s.wrappedBuf;
@@ -237,8 +212,8 @@ public class GenericUDAFLastValue extend
wrappedEval.iterate(lb, parameters);
}
- Object o = ObjectInspectorUtils.copyToStandardObject(parameters[0],
- inputOI(), ObjectInspectorCopyOption.WRITABLE);
+ Object o = ObjectInspectorUtils.copyToStandardObject(parameters[0], inputOI(),
+ ObjectInspectorCopyOption.WRITABLE);
if (!lb.skipNulls || o != null) {
s.lastValue = o;
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFNTile.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFNTile.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFNTile.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFNTile.java Tue Sep 2 16:30:33 2014
@@ -38,144 +38,129 @@ import org.apache.hadoop.hive.serde2.typ
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
import org.apache.hadoop.io.IntWritable;
-@WindowFunctionDescription
-(
- description = @Description(
- name = "rank",
- value = "_FUNC_(x) NTILE allows easy calculation of tertiles, quartiles, deciles and other " +
- "common summary statistics. This function divides an ordered partition into a specified " +
- "number of groups called buckets and assigns a bucket number to each row in the partition."
- ),
- supportsWindow = false,
- pivotResult = true
+@WindowFunctionDescription(
+ description = @Description(
+ name = "rank",
+ value = "_FUNC_(x) NTILE allows easy calculation of tertiles, quartiles, deciles and other "
+ +"common summary statistics. This function divides an ordered partition into a "
+ + "specified number of groups called buckets and assigns a bucket number to each row "
+ + "in the partition."
+ ),
+ supportsWindow = false,
+ pivotResult = true
)
-public class GenericUDAFNTile extends AbstractGenericUDAFResolver
-{
- static final Log LOG = LogFactory.getLog(GenericUDAFNTile.class.getName());
-
- @Override
- public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException
- {
- if (parameters.length != 1)
- {
- throw new UDFArgumentTypeException(parameters.length - 1, "Exactly one argument is expected.");
- }
- ObjectInspector oi = TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(parameters[0]);
-
- boolean c = ObjectInspectorUtils.compareTypes(oi, PrimitiveObjectInspectorFactory.writableIntObjectInspector);
- if (!c)
- {
- throw new UDFArgumentTypeException(0, "Number of tiles must be an int expression");
- }
-
- return new GenericUDAFNTileEvaluator();
- }
-
- static class NTileBuffer implements AggregationBuffer
- {
- Integer numBuckets;
- int numRows;
-
- void init()
- {
- numBuckets = null;
- numRows = 0;
- }
-
- NTileBuffer()
- {
- init();
- }
- }
-
- public static class GenericUDAFNTileEvaluator extends GenericUDAFEvaluator
- {
- private transient PrimitiveObjectInspector inputOI;
-
- @Override
- public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException
- {
- assert (parameters.length == 1);
- super.init(m, parameters);
- if (m != Mode.COMPLETE)
- {
- throw new HiveException(
- "Only COMPLETE mode supported for NTile function");
- }
- inputOI = (PrimitiveObjectInspector) parameters[0];
- return ObjectInspectorFactory.getStandardListObjectInspector(PrimitiveObjectInspectorFactory.writableIntObjectInspector);
- }
-
- @Override
- public AggregationBuffer getNewAggregationBuffer() throws HiveException
- {
- return new NTileBuffer();
- }
-
- @Override
- public void reset(AggregationBuffer agg) throws HiveException
- {
- ((NTileBuffer) agg).init();
- }
-
- @Override
- public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException
- {
- NTileBuffer rb = (NTileBuffer) agg;
- if ( rb.numBuckets == null)
- {
- rb.numBuckets = PrimitiveObjectInspectorUtils.getInt(parameters[0], inputOI);
- }
- rb.numRows++;
- }
-
- @Override
- public Object terminatePartial(AggregationBuffer agg) throws HiveException
- {
- throw new HiveException("terminatePartial not supported");
- }
-
- @Override
- public void merge(AggregationBuffer agg, Object partial) throws HiveException
- {
- throw new HiveException("merge not supported");
- }
-
- @Override
- public Object terminate(AggregationBuffer agg) throws HiveException
- {
- NTileBuffer rb = (NTileBuffer) agg;
- ArrayList<IntWritable> res = new ArrayList<IntWritable>(rb.numRows);
-
- /*
- * if there is a remainder from numRows/numBuckets; then distribute increase the size of the first 'rem' buckets by 1.
- */
-
- int bucketsz = rb.numRows / rb.numBuckets;
- int rem = rb.numRows % rb.numBuckets;
- int start = 0;
- int bucket = 1;
- while ( start < rb.numRows)
- {
- int end = start + bucketsz;
- if (rem > 0)
- {
- end++; rem--;
- }
- end = Math.min(rb.numRows, end);
- for(int i = start; i < end; i++)
- {
- res.add(new IntWritable(bucket));
- }
- start = end;
- bucket++;
- }
+public class GenericUDAFNTile extends AbstractGenericUDAFResolver {
- return res;
- }
+ static final Log LOG = LogFactory.getLog(GenericUDAFNTile.class.getName());
- }
+ @Override
+ public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
+ if (parameters.length != 1) {
+ throw new UDFArgumentTypeException(parameters.length - 1,
+ "Exactly one argument is expected.");
+ }
+ ObjectInspector oi = TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(parameters[0]);
+
+ boolean c = ObjectInspectorUtils.compareTypes(oi,
+ PrimitiveObjectInspectorFactory.writableIntObjectInspector);
+ if (!c) {
+ throw new UDFArgumentTypeException(0, "Number of tiles must be an int expression");
+ }
+
+ return new GenericUDAFNTileEvaluator();
+ }
+
+ static class NTileBuffer implements AggregationBuffer {
+
+ Integer numBuckets;
+ int numRows;
+
+ void init() {
+ numBuckets = null;
+ numRows = 0;
+ }
+
+ NTileBuffer() {
+ init();
+ }
+ }
+
+ public static class GenericUDAFNTileEvaluator extends GenericUDAFEvaluator {
+
+ private transient PrimitiveObjectInspector inputOI;
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ super.init(m, parameters);
+ if (m != Mode.COMPLETE) {
+ throw new HiveException("Only COMPLETE mode supported for NTile function");
+ }
+ inputOI = (PrimitiveObjectInspector) parameters[0];
+ return ObjectInspectorFactory.getStandardListObjectInspector(
+ PrimitiveObjectInspectorFactory.writableIntObjectInspector);
+ }
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ return new NTileBuffer();
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ ((NTileBuffer) agg).init();
+ }
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ NTileBuffer rb = (NTileBuffer) agg;
+ if (rb.numBuckets == null) {
+ rb.numBuckets = PrimitiveObjectInspectorUtils.getInt(parameters[0], inputOI);
+ }
+ rb.numRows++;
+ }
+
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ throw new HiveException("terminatePartial not supported");
+ }
+
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ throw new HiveException("merge not supported");
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ NTileBuffer rb = (NTileBuffer) agg;
+ ArrayList<IntWritable> res = new ArrayList<IntWritable>(rb.numRows);
+
+ /*
+ * if there is a remainder from numRows/numBuckets; then distribute increase the size of the first 'rem' buckets by 1.
+ */
+
+ int bucketsz = rb.numRows / rb.numBuckets;
+ int rem = rb.numRows % rb.numBuckets;
+ int start = 0;
+ int bucket = 1;
+ while (start < rb.numRows) {
+ int end = start + bucketsz;
+ if (rem > 0) {
+ end++;
+ rem--;
+ }
+ end = Math.min(rb.numRows, end);
+ for (int i = start; i < end; i++) {
+ res.add(new IntWritable(bucket));
+ }
+ start = end;
+ bucket++;
+ }
+ return res;
+ }
+
+ }
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFPercentRank.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFPercentRank.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFPercentRank.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFPercentRank.java Tue Sep 2 16:30:33 2014
@@ -31,56 +31,52 @@ import org.apache.hadoop.hive.serde2.obj
import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
import org.apache.hadoop.io.IntWritable;
-@WindowFunctionDescription
-(
- description = @Description(
- name = "percent_rank",
- value = "_FUNC_(x) PERCENT_RANK is similar to CUME_DIST, but it uses rank values rather " +
- "than row counts in its numerator. PERCENT_RANK of a row is calculated as: " +
- "(rank of row in its partition - 1) / (number of rows in the partition - 1)"
- ),
- supportsWindow = false,
- pivotResult = true,
- rankingFunction = true,
- impliesOrder = true
+@WindowFunctionDescription(
+ description = @Description(
+ name = "percent_rank",
+ value = "_FUNC_(x) PERCENT_RANK is similar to CUME_DIST, but it uses rank values rather " +
+ "than row counts in its numerator. PERCENT_RANK of a row is calculated as: " +
+ "(rank of row in its partition - 1) / (number of rows in the partition - 1)"
+ ),
+ supportsWindow = false,
+ pivotResult = true,
+ rankingFunction = true,
+ impliesOrder = true
)
-public class GenericUDAFPercentRank extends GenericUDAFRank
-{
- static final Log LOG = LogFactory.getLog(GenericUDAFPercentRank.class.getName());
-
- @Override
- protected GenericUDAFAbstractRankEvaluator createEvaluator()
- {
- return new GenericUDAFPercentRankEvaluator();
- }
-
- public static class GenericUDAFPercentRankEvaluator extends GenericUDAFAbstractRankEvaluator
- {
- @Override
- public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException
- {
- super.init(m, parameters);
- return ObjectInspectorFactory.getStandardListObjectInspector(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
- }
-
- @Override
- public Object terminate(AggregationBuffer agg) throws HiveException
- {
- ArrayList<IntWritable> ranks = ((RankBuffer) agg).rowNums;
- double sz = ranks.size();
- if ( sz > 1 ) {
+public class GenericUDAFPercentRank extends GenericUDAFRank {
+
+ static final Log LOG = LogFactory.getLog(GenericUDAFPercentRank.class.getName());
+
+ @Override
+ protected GenericUDAFAbstractRankEvaluator createEvaluator() {
+ return new GenericUDAFPercentRankEvaluator();
+ }
+
+ public static class GenericUDAFPercentRankEvaluator extends GenericUDAFAbstractRankEvaluator {
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ super.init(m, parameters);
+ return ObjectInspectorFactory.getStandardListObjectInspector(
+ PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ ArrayList<IntWritable> ranks = ((RankBuffer) agg).rowNums;
+ double sz = ranks.size();
+ if (sz > 1) {
sz = sz - 1;
}
- ArrayList<DoubleWritable> pranks = new ArrayList<DoubleWritable>(ranks.size());
+ ArrayList<DoubleWritable> pranks = new ArrayList<DoubleWritable>(ranks.size());
+
+ for (IntWritable i : ranks) {
+ double pr = ((double) i.get() - 1) / sz;
+ pranks.add(new DoubleWritable(pr));
+ }
- for(IntWritable i : ranks)
- {
- double pr = ((double)i.get() - 1)/sz;
- pranks.add(new DoubleWritable(pr));
- }
-
- return pranks;
- }
- }
+ return pranks;
+ }
+ }
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRank.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRank.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRank.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRank.java Tue Sep 2 16:30:33 2014
@@ -38,170 +38,150 @@ import org.apache.hadoop.hive.serde2.typ
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
import org.apache.hadoop.io.IntWritable;
-@WindowFunctionDescription
-(
- description = @Description(
- name = "rank",
- value = "_FUNC_(x)"
- ),
- supportsWindow = false,
- pivotResult = true,
- rankingFunction = true,
- impliesOrder = true
-)
-public class GenericUDAFRank extends AbstractGenericUDAFResolver
-{
- static final Log LOG = LogFactory.getLog(GenericUDAFRank.class.getName());
-
- @Override
- public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException
- {
- if (parameters.length < 1)
- {
- throw new UDFArgumentTypeException(parameters.length - 1, "One or more arguments are expected.");
- }
- for(int i=0; i<parameters.length; i++)
- {
- ObjectInspector oi = TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(parameters[i]);
- if (!ObjectInspectorUtils.compareSupported(oi))
- {
- throw new UDFArgumentTypeException(i,
- "Cannot support comparison of map<> type or complex type containing map<>.");
- }
- }
- return createEvaluator();
- }
-
- protected GenericUDAFAbstractRankEvaluator createEvaluator()
- {
- return new GenericUDAFRankEvaluator();
- }
-
- static class RankBuffer implements AggregationBuffer
- {
- ArrayList<IntWritable> rowNums;
- int currentRowNum;
- Object[] currVal;
- int currentRank;
- int numParams;
- boolean supportsStreaming;
-
- RankBuffer(int numParams, boolean supportsStreaming)
- {
- this.numParams = numParams;
- this.supportsStreaming = supportsStreaming;
- init();
- }
-
- void init()
- {
- rowNums = new ArrayList<IntWritable>();
- currentRowNum = 0;
- currentRank = 0;
- currVal = new Object[numParams];
- if ( supportsStreaming ) {
- /* initialize rowNums to have 1 row */
- rowNums.add(null);
- }
- }
-
- void incrRowNum() { currentRowNum++; }
-
- void addRank()
- {
- if ( supportsStreaming ) {
- rowNums.set(0, new IntWritable(currentRank));
- } else {
- rowNums.add(new IntWritable(currentRank));
- }
- }
- }
-
- public static abstract class GenericUDAFAbstractRankEvaluator extends GenericUDAFEvaluator
- {
- ObjectInspector[] inputOI;
- ObjectInspector[] outputOI;
- boolean isStreamingMode = false;
-
- protected boolean isStreaming() {
- return isStreamingMode;
- }
-
- @Override
- public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException
- {
- super.init(m, parameters);
- if (m != Mode.COMPLETE)
- {
- throw new HiveException(
- "Only COMPLETE mode supported for Rank function");
- }
- inputOI = parameters;
- outputOI = new ObjectInspector[inputOI.length];
- for(int i=0; i < inputOI.length; i++)
- {
- outputOI[i] = ObjectInspectorUtils.getStandardObjectInspector(inputOI[i], ObjectInspectorCopyOption.JAVA);
- }
- return ObjectInspectorFactory.getStandardListObjectInspector(PrimitiveObjectInspectorFactory.writableIntObjectInspector);
- }
-
- @Override
- public AggregationBuffer getNewAggregationBuffer() throws HiveException
- {
- return new RankBuffer(inputOI.length, isStreamingMode);
- }
-
- @Override
- public void reset(AggregationBuffer agg) throws HiveException
- {
- ((RankBuffer) agg).init();
- }
-
- @Override
- public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException
- {
- RankBuffer rb = (RankBuffer) agg;
- int c = GenericUDAFRank.compare(rb.currVal, outputOI, parameters, inputOI);
- rb.incrRowNum();
- if ( rb.currentRowNum == 1 || c != 0 )
- {
- nextRank(rb);
- rb.currVal = GenericUDAFRank.copyToStandardObject(parameters, inputOI, ObjectInspectorCopyOption.JAVA);
- }
- rb.addRank();
- }
-
- /*
- * Called when the value in the partition has changed. Update the currentRank
- */
- protected void nextRank(RankBuffer rb)
- {
- rb.currentRank = rb.currentRowNum;
- }
-
- @Override
- public Object terminatePartial(AggregationBuffer agg) throws HiveException
- {
- throw new HiveException("terminatePartial not supported");
- }
-
- @Override
- public void merge(AggregationBuffer agg, Object partial) throws HiveException
- {
- throw new HiveException("merge not supported");
- }
-
- @Override
- public Object terminate(AggregationBuffer agg) throws HiveException
- {
- return ((RankBuffer) agg).rowNums;
- }
-
- }
-
- public static class GenericUDAFRankEvaluator extends
- GenericUDAFAbstractRankEvaluator implements
- ISupportStreamingModeForWindowing {
+@WindowFunctionDescription(
+ description = @Description(
+ name = "rank",
+ value = "_FUNC_(x)"),
+ supportsWindow = false,
+ pivotResult = true,
+ rankingFunction = true,
+ impliesOrder = true)
+public class GenericUDAFRank extends AbstractGenericUDAFResolver {
+
+ static final Log LOG = LogFactory.getLog(GenericUDAFRank.class.getName());
+
+ @Override
+ public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
+ if (parameters.length < 1) {
+ throw new UDFArgumentTypeException(parameters.length - 1,
+ "One or more arguments are expected.");
+ }
+ for (int i = 0; i < parameters.length; i++) {
+ ObjectInspector oi = TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(parameters[i]);
+ if (!ObjectInspectorUtils.compareSupported(oi)) {
+ throw new UDFArgumentTypeException(i,
+ "Cannot support comparison of map<> type or complex type containing map<>.");
+ }
+ }
+ return createEvaluator();
+ }
+
+ protected GenericUDAFAbstractRankEvaluator createEvaluator() {
+ return new GenericUDAFRankEvaluator();
+ }
+
+ static class RankBuffer implements AggregationBuffer {
+
+ ArrayList<IntWritable> rowNums;
+ int currentRowNum;
+ Object[] currVal;
+ int currentRank;
+ int numParams;
+ boolean supportsStreaming;
+
+ RankBuffer(int numParams, boolean supportsStreaming) {
+ this.numParams = numParams;
+ this.supportsStreaming = supportsStreaming;
+ init();
+ }
+
+ void init() {
+ rowNums = new ArrayList<IntWritable>();
+ currentRowNum = 0;
+ currentRank = 0;
+ currVal = new Object[numParams];
+ if (supportsStreaming) {
+ /* initialize rowNums to have 1 row */
+ rowNums.add(null);
+ }
+ }
+
+ void incrRowNum() { currentRowNum++; }
+
+ void addRank() {
+ if (supportsStreaming) {
+ rowNums.set(0, new IntWritable(currentRank));
+ } else {
+ rowNums.add(new IntWritable(currentRank));
+ }
+ }
+ }
+
+ public static abstract class GenericUDAFAbstractRankEvaluator extends GenericUDAFEvaluator {
+
+ ObjectInspector[] inputOI;
+ ObjectInspector[] outputOI;
+ boolean isStreamingMode = false;
+
+ protected boolean isStreaming() {
+ return isStreamingMode;
+ }
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ super.init(m, parameters);
+ if (m != Mode.COMPLETE) {
+ throw new HiveException("Only COMPLETE mode supported for Rank function");
+ }
+ inputOI = parameters;
+ outputOI = new ObjectInspector[inputOI.length];
+ for (int i = 0; i < inputOI.length; i++) {
+ outputOI[i] = ObjectInspectorUtils.getStandardObjectInspector(inputOI[i],
+ ObjectInspectorCopyOption.JAVA);
+ }
+ return ObjectInspectorFactory.getStandardListObjectInspector(
+ PrimitiveObjectInspectorFactory.writableIntObjectInspector);
+ }
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ return new RankBuffer(inputOI.length, isStreamingMode);
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ ((RankBuffer) agg).init();
+ }
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ RankBuffer rb = (RankBuffer) agg;
+ int c = GenericUDAFRank.compare(rb.currVal, outputOI, parameters, inputOI);
+ rb.incrRowNum();
+ if (rb.currentRowNum == 1 || c != 0) {
+ nextRank(rb);
+ rb.currVal =
+ GenericUDAFRank.copyToStandardObject(parameters, inputOI, ObjectInspectorCopyOption.JAVA);
+ }
+ rb.addRank();
+ }
+
+ /*
+ * Called when the value in the partition has changed. Update the currentRank
+ */
+ protected void nextRank(RankBuffer rb) {
+ rb.currentRank = rb.currentRowNum;
+ }
+
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ throw new HiveException("terminatePartial not supported");
+ }
+
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ throw new HiveException("merge not supported");
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ return ((RankBuffer) agg).rowNums;
+ }
+
+ }
+
+ public static class GenericUDAFRankEvaluator extends GenericUDAFAbstractRankEvaluator
+ implements ISupportStreamingModeForWindowing {
@Override
public Object getNextResult(AggregationBuffer agg) throws HiveException {
@@ -215,18 +195,15 @@ public class GenericUDAFRank extends Abs
}
@Override
- public int getRowsRemainingAfterTerminate()
- throws HiveException {
+ public int getRowsRemainingAfterTerminate() throws HiveException {
return 0;
}
}
public static int compare(Object[] o1, ObjectInspector[] oi1, Object[] o2,
- ObjectInspector[] oi2)
- {
+ ObjectInspector[] oi2) {
int c = 0;
- for (int i = 0; i < oi1.length; i++)
- {
+ for (int i = 0; i < oi1.length; i++) {
c = ObjectInspectorUtils.compare(o1[i], oi1[i], o2[i], oi2[i]);
if (c != 0) {
return c;
@@ -235,15 +212,11 @@ public class GenericUDAFRank extends Abs
return c;
}
- public static Object[] copyToStandardObject(Object[] o,
- ObjectInspector[] oi,
- ObjectInspectorCopyOption objectInspectorOption)
- {
+ public static Object[] copyToStandardObject(Object[] o, ObjectInspector[] oi,
+ ObjectInspectorCopyOption objectInspectorOption) {
Object[] out = new Object[o.length];
- for (int i = 0; i < oi.length; i++)
- {
- out[i] = ObjectInspectorUtils.copyToStandardObject(o[i], oi[i],
- objectInspectorOption);
+ for (int i = 0; i < oi.length; i++) {
+ out[i] = ObjectInspectorUtils.copyToStandardObject(o[i], oi[i], objectInspectorOption);
}
return out;
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRowNumber.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRowNumber.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRowNumber.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRowNumber.java Tue Sep 2 16:30:33 2014
@@ -34,110 +34,89 @@ import org.apache.hadoop.hive.serde2.obj
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
import org.apache.hadoop.io.IntWritable;
-@WindowFunctionDescription
-(
- description = @Description(
- name = "row_number",
- value = "_FUNC_() - The ROW_NUMBER function assigns a unique number (sequentially, starting from 1, as defined by ORDER BY) to each row within the partition."
- ),
- supportsWindow = false,
- pivotResult = true
+@WindowFunctionDescription(
+ description = @Description(
+ name = "row_number",
+ value = "_FUNC_() - The ROW_NUMBER function assigns a unique number (sequentially, starting "
+ + "from 1, as defined by ORDER BY) to each row within the partition."
+ ),
+ supportsWindow = false,
+ pivotResult = true
)
-public class GenericUDAFRowNumber extends AbstractGenericUDAFResolver
-{
- static final Log LOG = LogFactory.getLog(GenericUDAFRowNumber.class.getName());
-
- @Override
- public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
- throws SemanticException
- {
- if (parameters.length != 0)
- {
- throw new UDFArgumentTypeException(parameters.length - 1,
- "No argument is expected.");
- }
- return new GenericUDAFRowNumberEvaluator();
- }
-
- static class RowNumberBuffer implements AggregationBuffer
- {
- ArrayList<IntWritable> rowNums;
- int nextRow;
-
- void init()
- {
- rowNums = new ArrayList<IntWritable>();
- }
-
- RowNumberBuffer()
- {
- init();
- nextRow = 1;
- }
-
- void incr()
- {
- rowNums.add(new IntWritable(nextRow++));
- }
- }
-
- public static class GenericUDAFRowNumberEvaluator extends
- GenericUDAFEvaluator
- {
-
- @Override
- public ObjectInspector init(Mode m, ObjectInspector[] parameters)
- throws HiveException
- {
- super.init(m, parameters);
- if (m != Mode.COMPLETE)
- {
- throw new HiveException("Only COMPLETE mode supported for row_number function");
- }
-
- return ObjectInspectorFactory.getStandardListObjectInspector(
- PrimitiveObjectInspectorFactory.writableIntObjectInspector);
- }
-
- @Override
- public AggregationBuffer getNewAggregationBuffer() throws HiveException
- {
- return new RowNumberBuffer();
- }
-
- @Override
- public void reset(AggregationBuffer agg) throws HiveException
- {
- ((RowNumberBuffer) agg).init();
- }
-
- @Override
- public void iterate(AggregationBuffer agg, Object[] parameters)
- throws HiveException
- {
- ((RowNumberBuffer) agg).incr();
- }
-
- @Override
- public Object terminatePartial(AggregationBuffer agg)
- throws HiveException
- {
- throw new HiveException("terminatePartial not supported");
- }
-
- @Override
- public void merge(AggregationBuffer agg, Object partial)
- throws HiveException
- {
- throw new HiveException("merge not supported");
- }
-
- @Override
- public Object terminate(AggregationBuffer agg) throws HiveException
- {
- return ((RowNumberBuffer) agg).rowNums;
- }
+public class GenericUDAFRowNumber extends AbstractGenericUDAFResolver {
- }
+ static final Log LOG = LogFactory.getLog(GenericUDAFRowNumber.class.getName());
+
+ @Override
+ public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
+ if (parameters.length != 0) {
+ throw new UDFArgumentTypeException(parameters.length - 1, "No argument is expected.");
+ }
+ return new GenericUDAFRowNumberEvaluator();
+ }
+
+ static class RowNumberBuffer implements AggregationBuffer {
+
+ ArrayList<IntWritable> rowNums;
+ int nextRow;
+
+ void init() {
+ rowNums = new ArrayList<IntWritable>();
+ }
+
+ RowNumberBuffer() {
+ init();
+ nextRow = 1;
+ }
+
+ void incr() {
+ rowNums.add(new IntWritable(nextRow++));
+ }
+ }
+
+ public static class GenericUDAFRowNumberEvaluator extends GenericUDAFEvaluator {
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ super.init(m, parameters);
+ if (m != Mode.COMPLETE) {
+ throw new HiveException("Only COMPLETE mode supported for row_number function");
+ }
+
+ return ObjectInspectorFactory.getStandardListObjectInspector(
+ PrimitiveObjectInspectorFactory.writableIntObjectInspector);
+ }
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ return new RowNumberBuffer();
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ ((RowNumberBuffer) agg).init();
+ }
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ ((RowNumberBuffer) agg).incr();
+ }
+
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ throw new HiveException("terminatePartial not supported");
+ }
+
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ throw new HiveException("merge not supported");
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ return ((RowNumberBuffer) agg).rowNums;
+ }
+
+ }
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBasePad.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBasePad.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBasePad.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBasePad.java Tue Sep 2 16:30:33 2014
@@ -46,7 +46,7 @@ public abstract class GenericUDFBasePad
public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
if (arguments.length != 3) {
throw new UDFArgumentException(udfName + " requires three arguments. Found :"
- + arguments.length);
+ + arguments.length);
}
converter1 = checkTextArguments(arguments, 0);
converter2 = checkIntArguments(arguments, 1);
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseTrim.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseTrim.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseTrim.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseTrim.java Tue Sep 2 16:30:33 2014
@@ -40,14 +40,14 @@ public abstract class GenericUDFBaseTrim
public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
if (arguments.length != 1) {
throw new UDFArgumentException(udfName + " requires one value argument. Found :"
- + arguments.length);
+ + arguments.length);
}
PrimitiveObjectInspector argumentOI;
if(arguments[0] instanceof PrimitiveObjectInspector) {
argumentOI = (PrimitiveObjectInspector) arguments[0];
} else {
throw new UDFArgumentException(udfName + " takes only primitive types. found "
- + arguments[0].getTypeName());
+ + arguments[0].getTypeName());
}
switch (argumentOI.getPrimitiveCategory()) {
case STRING:
@@ -56,7 +56,7 @@ public abstract class GenericUDFBaseTrim
break;
default:
throw new UDFArgumentException(udfName + " takes only STRING/CHAR/VARCHAR types. Found "
- + argumentOI.getPrimitiveCategory());
+ + argumentOI.getPrimitiveCategory());
}
converter = new TextConverter(argumentOI);
return PrimitiveObjectInspectorFactory.writableStringObjectInspector;
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBetween.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBetween.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBetween.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBetween.java Tue Sep 2 16:30:33 2014
@@ -51,7 +51,7 @@ public class GenericUDFBetween extends G
@Override
public Object evaluate(DeferredObject[] arguments) throws HiveException {
boolean invert = (Boolean) ((PrimitiveObjectInspector) argumentOIs[0])
- .getPrimitiveJavaObject(arguments[0].get());
+ .getPrimitiveJavaObject(arguments[0].get());
BooleanWritable left = ((BooleanWritable)egt.evaluate(new DeferredObject[] {arguments[1], arguments[2]}));
if (left == null) {
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDateDiff.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDateDiff.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDateDiff.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDateDiff.java Tue Sep 2 16:30:33 2014
@@ -68,7 +68,7 @@ public class GenericUDFDateDiff extends
private transient PrimitiveCategory inputType1;
private transient PrimitiveCategory inputType2;
private IntWritable result = new IntWritable();
-
+
public GenericUDFDateDiff() {
formatter.setTimeZone(TimeZone.getTimeZone("UTC"));
}
@@ -89,8 +89,8 @@ public class GenericUDFDateDiff extends
@Override
public IntWritable evaluate(DeferredObject[] arguments) throws HiveException {
- output = evaluate(convertToDate(inputType1, inputConverter1, arguments[0]),
- convertToDate(inputType2, inputConverter2, arguments[1]));
+ output = evaluate(convertToDate(inputType1, inputConverter1, arguments[0]),
+ convertToDate(inputType2, inputConverter2, arguments[1]));
return output;
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDecode.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDecode.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDecode.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDecode.java Tue Sep 2 16:30:33 2014
@@ -44,8 +44,8 @@ import org.apache.hadoop.io.Text;
@Description(name = "decode",
value = "_FUNC_(bin, str) - Decode the first argument using the second argument character set",
extended = "Possible options for the character set are 'US_ASCII', 'ISO-8859-1',\n" +
- "'UTF-8', 'UTF-16BE', 'UTF-16LE', and 'UTF-16'. If either argument\n" +
- "is null, the result will also be null")
+ "'UTF-8', 'UTF-16BE', 'UTF-16LE', and 'UTF-16'. If either argument\n" +
+ "is null, the result will also be null")
public class GenericUDFDecode extends GenericUDF {
private transient CharsetDecoder decoder = null;
private transient BinaryObjectInspector bytesOI = null;
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLpad.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLpad.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLpad.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLpad.java Tue Sep 2 16:30:33 2014
@@ -46,7 +46,7 @@ public class GenericUDFLpad extends Gene
// Copy the padding
for (int i = 0; i < pos; i += pad.getLength()) {
for (int j = 0; j < pad.getLength() && j < pos - i; j++) {
- data[i + j] = padTxt[j];
+ data[i + j] = padTxt[j];
}
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRpad.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRpad.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRpad.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRpad.java Tue Sep 2 16:30:33 2014
@@ -48,7 +48,7 @@ public class GenericUDFRpad extends Gene
// Copy the padding
while (pos < len) {
for (int i = 0; i < pad.getLength() && i < len - pos; i++) {
- data[pos + i] = padTxt[i];
+ data[pos + i] = padTxt[i];
}
pos += pad.getLength();
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToBinary.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToBinary.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToBinary.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToBinary.java Tue Sep 2 16:30:33 2014
@@ -59,7 +59,7 @@ public class GenericUDFToBinary extends
|| (argumentOI instanceof StringObjectInspector)
|| (argumentOI instanceof VoidObjectInspector))){
throw new UDFArgumentException("Only string, char, varchar or binary data can be cast into binary " +
- "data types.");
+ "data types.");
}
baConverter = new BinaryConverter(argumentOI,
PrimitiveObjectInspectorFactory.writableBinaryObjectInspector);
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFJSONTuple.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFJSONTuple.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFJSONTuple.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFJSONTuple.java Tue Sep 2 16:30:33 2014
@@ -47,7 +47,7 @@ import org.codehaus.jackson.type.JavaTyp
*/
@Description(name = "json_tuple",
value = "_FUNC_(jsonStr, p1, p2, ..., pn) - like get_json_object, but it takes multiple names and return a tuple. " +
- "All the input parameters and output column types are string.")
+ "All the input parameters and output column types are string.")
public class GenericUDTFJSONTuple extends GenericUDTF {
@@ -106,7 +106,7 @@ public class GenericUDTFJSONTuple extend
if (numCols < 1) {
throw new UDFArgumentException("json_tuple() takes at least two arguments: " +
- "the json string and a path expression");
+ "the json string and a path expression");
}
for (int i = 0; i < args.length; ++i) {
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFParseUrlTuple.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFParseUrlTuple.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFParseUrlTuple.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFParseUrlTuple.java Tue Sep 2 16:30:33 2014
@@ -86,7 +86,7 @@ public class GenericUDTFParseUrlTuple ex
if (numCols < 1) {
throw new UDFArgumentException("parse_url_tuple() takes at least two arguments: " +
- "the url string and a part name");
+ "the url string and a part name");
}
for (int i = 0; i < args.length; ++i) {
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/MatchPath.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/MatchPath.java?rev=1622060&r1=1622059&r2=1622060&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/MatchPath.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/MatchPath.java Tue Sep 2 16:30:33 2014
@@ -225,7 +225,7 @@ public class MatchPath extends TableFunc
if ( symbolArgsSz % 2 != 0)
{
throwErrorWithSignature("Symbol Name, Expression need to be specified in pairs: " +
- "there are odd number of symbol args");
+ "there are odd number of symbol args");
}
evaluator.symInfo = new SymbolsInfo(symbolArgsSz/2);
@@ -253,7 +253,7 @@ public class MatchPath extends TableFunc
PrimitiveObjectInspector.PrimitiveCategory.BOOLEAN )
{
throwErrorWithSignature(String.format("Currently a Symbol Expression(%s) " +
- "must be a boolean expression", symolExprArg.getExpressionTreeString()));
+ "must be a boolean expression", symolExprArg.getExpressionTreeString()));
}
evaluator.symInfo.add(symbolName, symolExprArg);
}