Return-Path: X-Original-To: apmail-hive-commits-archive@www.apache.org Delivered-To: apmail-hive-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 5D2EF17D32 for ; Sat, 28 Mar 2015 00:22:45 +0000 (UTC) Received: (qmail 46111 invoked by uid 500); 28 Mar 2015 00:22:37 -0000 Delivered-To: apmail-hive-commits-archive@hive.apache.org Received: (qmail 46041 invoked by uid 500); 28 Mar 2015 00:22:37 -0000 Mailing-List: contact commits-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hive-dev@hive.apache.org Delivered-To: mailing list commits@hive.apache.org Received: (qmail 44780 invoked by uid 99); 28 Mar 2015 00:22:36 -0000 Received: from eris.apache.org (HELO hades.apache.org) (140.211.11.105) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 28 Mar 2015 00:22:36 +0000 Received: from hades.apache.org (localhost [127.0.0.1]) by hades.apache.org (ASF Mail Server at hades.apache.org) with ESMTP id 5194AAC0E0C for ; Sat, 28 Mar 2015 00:22:36 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1669718 [28/29] - in /hive/branches/llap: ./ ant/src/org/apache/hadoop/hive/ant/ common/src/java/org/apache/hadoop/hive/common/ common/src/java/org/apache/hadoop/hive/common/type/ common/src/java/org/apache/hadoop/hive/conf/ common/src/jav... Date: Sat, 28 Mar 2015 00:22:27 -0000 To: commits@hive.apache.org From: sershe@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20150328002236.5194AAC0E0C@hades.apache.org> Modified: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java?rev=1669718&r1=1669717&r2=1669718&view=diff ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java (original) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java Sat Mar 28 00:22:15 2015 @@ -512,15 +512,6 @@ public class TimestampWritable implement return decimal != 0; } - /** - * Interprets a float as a unix timestamp and returns a Timestamp object - * @param f - * @return the equivalent Timestamp object - */ - public static Timestamp floatToTimestamp(float f) { - return doubleToTimestamp((double) f); - } - public static Timestamp decimalToTimestamp(HiveDecimal d) { BigDecimal nanoInstant = d.bigDecimalValue().multiply(BILLION_BIG_DECIMAL); int nanos = nanoInstant.remainder(BILLION_BIG_DECIMAL).intValue(); @@ -535,6 +526,15 @@ public class TimestampWritable implement return t; } + /** + * Converts the time in seconds to a timestamp. + * @param timeInSeconds time in seconds + * @return the timestamp + */ + public static Timestamp longToTimestamp(long timeInSeconds) { + return new Timestamp(timeInSeconds * 1000); + } + public static Timestamp doubleToTimestamp(double f) { long seconds = (long) f; Modified: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java?rev=1669718&r1=1669717&r2=1669718&view=diff ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java (original) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java Sat Mar 28 00:22:15 2015 @@ -35,6 +35,8 @@ import org.apache.hadoop.hive.serde2.laz import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyFloatObjectInspector; import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyHiveCharObjectInspector; import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyHiveDecimalObjectInspector; +import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyHiveIntervalYearMonthObjectInspector; +import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyHiveIntervalDayTimeObjectInspector; import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyHiveVarcharObjectInspector; import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyIntObjectInspector; import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyLongObjectInspector; @@ -125,6 +127,10 @@ public final class LazyFactory { return new LazyDate((LazyDateObjectInspector) oi); case TIMESTAMP: return new LazyTimestamp((LazyTimestampObjectInspector) oi); + case INTERVAL_YEAR_MONTH: + return new LazyHiveIntervalYearMonth((LazyHiveIntervalYearMonthObjectInspector) oi); + case INTERVAL_DAY_TIME: + return new LazyHiveIntervalDayTime((LazyHiveIntervalDayTimeObjectInspector) oi); case BINARY: return new LazyBinary((LazyBinaryObjectInspector) oi); case DECIMAL: Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveIntervalDayTime.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveIntervalDayTime.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveIntervalDayTime.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveIntervalDayTime.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,65 @@ +/** + * 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.serde2.lazy; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; +import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable; +import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyHiveIntervalDayTimeObjectInspector; +import org.apache.hadoop.io.Text; + +public class LazyHiveIntervalDayTime + extends LazyPrimitive { + + public LazyHiveIntervalDayTime(LazyHiveIntervalDayTimeObjectInspector oi) { + super(oi); + data = new HiveIntervalDayTimeWritable(); + } + + public LazyHiveIntervalDayTime(LazyHiveIntervalDayTime copy) { + super(copy); + data = new HiveIntervalDayTimeWritable(copy.data); + } + + @Override + public void init(ByteArrayRef bytes, int start, int length) { + String s = null; + try { + s = Text.decode(bytes.getData(), start, length); + data.set(HiveIntervalDayTime.valueOf(s)); + isNull = false; + } catch (Exception e) { + isNull = true; + logExceptionMessage(bytes, start, length, "INTERVAL_DAY_TIME"); + } + } + + public static void writeUTF8(OutputStream out, HiveIntervalDayTimeWritable i) throws IOException { + ByteBuffer b = Text.encode(i.toString()); + out.write(b.array(), 0, b.limit()); + } + + @Override + public HiveIntervalDayTimeWritable getWritableObject() { + return data; + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveIntervalYearMonth.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveIntervalYearMonth.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveIntervalYearMonth.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveIntervalYearMonth.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,66 @@ +/** + * 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.serde2.lazy; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; +import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable; +import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyHiveIntervalYearMonthObjectInspector; +import org.apache.hadoop.io.Text; + +public class LazyHiveIntervalYearMonth + extends LazyPrimitive { + + public LazyHiveIntervalYearMonth(LazyHiveIntervalYearMonthObjectInspector oi) { + super(oi); + data = new HiveIntervalYearMonthWritable(); + } + + public LazyHiveIntervalYearMonth(LazyHiveIntervalYearMonth copy) { + super(copy); + data = new HiveIntervalYearMonthWritable(copy.data); + } + + @Override + public void init(ByteArrayRef bytes, int start, int length) { + String s = null; + try { + s = Text.decode(bytes.getData(), start, length); + data.set(HiveIntervalYearMonth.valueOf(s)); + isNull = false; + } catch (Exception e) { + isNull = true; + logExceptionMessage(bytes, start, length, "INTERVAL_YEAR_MONTH"); + } + } + + public static void writeUTF8(OutputStream out, HiveIntervalYearMonthWritable i) + throws IOException { + ByteBuffer b = Text.encode(i.toString()); + out.write(b.array(), 0, b.limit()); + } + + @Override + public HiveIntervalYearMonthWritable getWritableObject() { + return data; + } +} Modified: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySimpleSerDe.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySimpleSerDe.java?rev=1669718&r1=1669717&r2=1669718&view=diff ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySimpleSerDe.java (original) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySimpleSerDe.java Sat Mar 28 00:22:15 2015 @@ -126,7 +126,8 @@ public class LazySimpleSerDe extends Abs // Create the ObjectInspectors for the fields cachedObjectInspector = LazyFactory.createLazyStructInspector(serdeParams - .getColumnNames(), serdeParams.getColumnTypes(), serdeParams); + .getColumnNames(), serdeParams.getColumnTypes(), + new LazyObjectInspectorParametersImpl(serdeParams)); cachedLazyStruct = (LazyStruct) LazyFactory .createLazyObject(cachedObjectInspector); Modified: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java?rev=1669718&r1=1669717&r2=1669718&view=diff ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java (original) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java Sat Mar 28 00:22:15 2015 @@ -39,6 +39,8 @@ import org.apache.hadoop.hive.serde2.obj import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveIntervalDayTimeObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveIntervalYearMonthObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector; @@ -254,6 +256,16 @@ public final class LazyUtils { ((TimestampObjectInspector) oi).getPrimitiveWritableObject(o)); break; } + case INTERVAL_YEAR_MONTH: { + LazyHiveIntervalYearMonth.writeUTF8(out, + ((HiveIntervalYearMonthObjectInspector) oi).getPrimitiveWritableObject(o)); + break; + } + case INTERVAL_DAY_TIME: { + LazyHiveIntervalDayTime.writeUTF8(out, + ((HiveIntervalDayTimeObjectInspector) oi).getPrimitiveWritableObject(o)); + break; + } case DECIMAL: { LazyHiveDecimal.writeUTF8(out, ((HiveDecimalObjectInspector) oi).getPrimitiveJavaObject(o)); Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyHiveIntervalDayTimeObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyHiveIntervalDayTimeObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyHiveIntervalDayTimeObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyHiveIntervalDayTimeObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -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.hadoop.hive.serde2.lazy.objectinspector.primitive; + +import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; +import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable; +import org.apache.hadoop.hive.serde2.lazy.LazyHiveIntervalDayTime; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveIntervalDayTimeObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; + +public class LazyHiveIntervalDayTimeObjectInspector + extends AbstractPrimitiveLazyObjectInspector + implements HiveIntervalDayTimeObjectInspector{ + + LazyHiveIntervalDayTimeObjectInspector() { + super(TypeInfoFactory.intervalDayTimeTypeInfo); + } + + @Override + public Object copyObject(Object o) { + return o == null ? null : new LazyHiveIntervalDayTime((LazyHiveIntervalDayTime) o); + } + + @Override + public HiveIntervalDayTime getPrimitiveJavaObject(Object o) { + return o == null ? null : ((LazyHiveIntervalDayTime) o).getWritableObject().getHiveIntervalDayTime(); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyHiveIntervalYearMonthObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyHiveIntervalYearMonthObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyHiveIntervalYearMonthObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyHiveIntervalYearMonthObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -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.hadoop.hive.serde2.lazy.objectinspector.primitive; + +import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; +import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable; +import org.apache.hadoop.hive.serde2.lazy.LazyHiveIntervalYearMonth; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveIntervalYearMonthObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; + +public class LazyHiveIntervalYearMonthObjectInspector + extends AbstractPrimitiveLazyObjectInspector + implements HiveIntervalYearMonthObjectInspector{ + + LazyHiveIntervalYearMonthObjectInspector() { + super(TypeInfoFactory.intervalYearMonthTypeInfo); + } + + @Override + public Object copyObject(Object o) { + return o == null ? null : new LazyHiveIntervalYearMonth((LazyHiveIntervalYearMonth) o); + } + + @Override + public HiveIntervalYearMonth getPrimitiveJavaObject(Object o) { + return o == null ? null : ((LazyHiveIntervalYearMonth) o).getWritableObject().getHiveIntervalYearMonth(); + } +} Modified: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java?rev=1669718&r1=1669717&r2=1669718&view=diff ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java (original) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java Sat Mar 28 00:22:15 2015 @@ -72,6 +72,10 @@ public final class LazyPrimitiveObjectIn new LazyDateObjectInspector(); public static final LazyTimestampObjectInspector LAZY_TIMESTAMP_OBJECT_INSPECTOR = new LazyTimestampObjectInspector(); + public static final LazyHiveIntervalYearMonthObjectInspector LAZY_INTERVAL_YEAR_MONTH_OBJECT_INSPECTOR = + new LazyHiveIntervalYearMonthObjectInspector(); + public static final LazyHiveIntervalDayTimeObjectInspector LAZY_INTERVAL_DAY_TIME_OBJECT_INSPECTOR = + new LazyHiveIntervalDayTimeObjectInspector(); public static final LazyBinaryObjectInspector LAZY_BINARY_OBJECT_INSPECTOR = new LazyBinaryObjectInspector(); @@ -108,6 +112,10 @@ public final class LazyPrimitiveObjectIn LAZY_DATE_OBJECT_INSPECTOR); cachedPrimitiveLazyObjectInspectors.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.TIMESTAMP_TYPE_NAME), LAZY_TIMESTAMP_OBJECT_INSPECTOR); + cachedPrimitiveLazyObjectInspectors.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME), + LAZY_INTERVAL_YEAR_MONTH_OBJECT_INSPECTOR); + cachedPrimitiveLazyObjectInspectors.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME), + LAZY_INTERVAL_DAY_TIME_OBJECT_INSPECTOR); cachedPrimitiveLazyObjectInspectors.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.BINARY_TYPE_NAME), LAZY_BINARY_OBJECT_INSPECTOR); } Modified: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryFactory.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryFactory.java?rev=1669718&r1=1669717&r2=1669718&view=diff ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryFactory.java (original) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryFactory.java Sat Mar 28 00:22:15 2015 @@ -29,6 +29,8 @@ import org.apache.hadoop.hive.serde2.obj import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory; import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableHiveCharObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableHiveIntervalDayTimeObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableHiveIntervalYearMonthObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableHiveVarcharObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableHiveDecimalObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableBinaryObjectInspector; @@ -84,6 +86,10 @@ public final class LazyBinaryFactory { return new LazyBinaryDate((WritableDateObjectInspector) oi); case TIMESTAMP: return new LazyBinaryTimestamp((WritableTimestampObjectInspector) oi); + case INTERVAL_YEAR_MONTH: + return new LazyBinaryHiveIntervalYearMonth((WritableHiveIntervalYearMonthObjectInspector) oi); + case INTERVAL_DAY_TIME: + return new LazyBinaryHiveIntervalDayTime((WritableHiveIntervalDayTimeObjectInspector) oi); case BINARY: return new LazyBinaryBinary((WritableBinaryObjectInspector) oi); case DECIMAL: Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveIntervalDayTime.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveIntervalDayTime.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveIntervalDayTime.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveIntervalDayTime.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,56 @@ +/** + * 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.serde2.lazybinary; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable; +import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef; +import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VInt; +import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VLong; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableHiveIntervalDayTimeObjectInspector; + +/** + * LazyBinaryHiveIntervalDayTime + * A LazyBinaryObject that encodes a HiveIntervalDayTime + */ +public class LazyBinaryHiveIntervalDayTime extends + LazyBinaryPrimitive{ + static final Log LOG = LogFactory.getLog(LazyBinaryHiveIntervalDayTime.class); + + /** + * Reusable member for decoding integer. + */ + VInt vInt = new LazyBinaryUtils.VInt(); + VLong vLong = new LazyBinaryUtils.VLong(); + + LazyBinaryHiveIntervalDayTime(WritableHiveIntervalDayTimeObjectInspector oi) { + super(oi); + data = new HiveIntervalDayTimeWritable(); + } + + LazyBinaryHiveIntervalDayTime(LazyBinaryHiveIntervalDayTime copy) { + super(copy); + data = new HiveIntervalDayTimeWritable(copy.data); + } + + @Override + public void init(ByteArrayRef bytes, int start, int length) { + data.setFromBytes(bytes.getData(), start, length, vInt, vLong); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveIntervalYearMonth.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveIntervalYearMonth.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveIntervalYearMonth.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveIntervalYearMonth.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,54 @@ +/** + * 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.serde2.lazybinary; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable; +import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef; +import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VInt; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableHiveIntervalYearMonthObjectInspector; + +/** + * LazyBinaryHiveIntervalYearMonth + * A LazyBinaryObject that encodes a HiveIntervalYearMonth + */ +public class LazyBinaryHiveIntervalYearMonth extends + LazyBinaryPrimitive{ + static final Log LOG = LogFactory.getLog(LazyBinaryHiveIntervalYearMonth.class); + + /** + * Reusable member for decoding integer. + */ + VInt vInt = new LazyBinaryUtils.VInt(); + + LazyBinaryHiveIntervalYearMonth(WritableHiveIntervalYearMonthObjectInspector oi) { + super(oi); + data = new HiveIntervalYearMonthWritable(); + } + + LazyBinaryHiveIntervalYearMonth(LazyBinaryHiveIntervalYearMonth copy) { + super(copy); + data = new HiveIntervalYearMonthWritable(copy.data); + } + + @Override + public void init(ByteArrayRef bytes, int start, int length) { + data.setFromBytes(bytes.getData(), start, length, vInt); + } +} Modified: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java?rev=1669718&r1=1669717&r2=1669718&view=diff ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java (original) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java Sat Mar 28 00:22:15 2015 @@ -36,6 +36,8 @@ import org.apache.hadoop.hive.serde2.Ser import org.apache.hadoop.hive.serde2.SerDeStats; import org.apache.hadoop.hive.serde2.io.DateWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable; +import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable; import org.apache.hadoop.hive.serde2.io.TimestampWritable; import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef; import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; @@ -54,6 +56,8 @@ import org.apache.hadoop.hive.serde2.obj import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveIntervalDayTimeObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveIntervalYearMonthObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector; @@ -428,6 +432,20 @@ public class LazyBinarySerDe extends Abs return; } + case INTERVAL_YEAR_MONTH: { + HiveIntervalYearMonthWritable intervalYearMonth = + ((HiveIntervalYearMonthObjectInspector) poi).getPrimitiveWritableObject(obj); + intervalYearMonth.writeToByteStream(byteStream); + return; + } + + case INTERVAL_DAY_TIME: { + HiveIntervalDayTimeWritable intervalDayTime = + ((HiveIntervalDayTimeObjectInspector) poi).getPrimitiveWritableObject(obj); + intervalDayTime.writeToByteStream(byteStream); + return; + } + case DECIMAL: { HiveDecimalObjectInspector bdoi = (HiveDecimalObjectInspector) poi; HiveDecimalWritable t = bdoi.getPrimitiveWritableObject(obj); Modified: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java?rev=1669718&r1=1669717&r2=1669718&view=diff ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java (original) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java Sat Mar 28 00:22:15 2015 @@ -210,6 +210,16 @@ public final class LazyBinaryUtils { recordInfo.elementOffset = 0; recordInfo.elementSize = TimestampWritable.getTotalLength(bytes, offset); break; + case INTERVAL_YEAR_MONTH: + recordInfo.elementOffset = 0; + recordInfo.elementSize = WritableUtils.decodeVIntSize(bytes[offset]); + break; + case INTERVAL_DAY_TIME: + recordInfo.elementOffset = 0; + int secondsSize = WritableUtils.decodeVIntSize(bytes[offset]); + int nanosSize = WritableUtils.decodeVIntSize(bytes[offset + secondsSize]); + recordInfo.elementSize = secondsSize + nanosSize; + break; case DECIMAL: // using vint instead of 4 bytes LazyBinaryUtils.readVInt(bytes, offset, vInt); Modified: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java?rev=1669718&r1=1669717&r2=1669718&view=diff ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java (original) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java Sat Mar 28 00:22:15 2015 @@ -33,6 +33,8 @@ import org.apache.hadoop.hive.serde2.obj import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableFloatObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableHiveCharObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableHiveDecimalObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableHiveIntervalDayTimeObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableHiveIntervalYearMonthObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableHiveVarcharObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableIntObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableLongObjectInspector; @@ -120,6 +122,14 @@ public final class ObjectInspectorConver return new PrimitiveObjectInspectorConverter.TimestampConverter( inputOI, (SettableTimestampObjectInspector) outputOI); + case INTERVAL_YEAR_MONTH: + return new PrimitiveObjectInspectorConverter.HiveIntervalYearMonthConverter( + inputOI, + (SettableHiveIntervalYearMonthObjectInspector) outputOI); + case INTERVAL_DAY_TIME: + return new PrimitiveObjectInspectorConverter.HiveIntervalDayTimeConverter( + inputOI, + (SettableHiveIntervalDayTimeObjectInspector) outputOI); case BINARY: return new PrimitiveObjectInspectorConverter.BinaryConverter( inputOI, Modified: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java?rev=1669718&r1=1669717&r2=1669718&view=diff ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java (original) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java Sat Mar 28 00:22:15 2015 @@ -34,6 +34,8 @@ import org.apache.hadoop.hive.serde2.Ser import org.apache.hadoop.hive.serde2.io.DateWritable; import org.apache.hadoop.hive.serde2.io.HiveCharWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable; +import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable; import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable; import org.apache.hadoop.hive.serde2.io.TimestampWritable; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; @@ -47,6 +49,8 @@ import org.apache.hadoop.hive.serde2.obj import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveIntervalDayTimeObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveIntervalYearMonthObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaStringObjectInspector; @@ -60,6 +64,8 @@ import org.apache.hadoop.hive.serde2.obj import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableFloatObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableHiveCharObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableHiveDecimalObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableHiveIntervalDayTimeObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableHiveIntervalYearMonthObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableHiveVarcharObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableIntObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableLongObjectInspector; @@ -542,6 +548,14 @@ public final class ObjectInspectorUtils TimestampWritable t = ((TimestampObjectInspector) poi) .getPrimitiveWritableObject(o); return t.hashCode(); + case INTERVAL_YEAR_MONTH: + HiveIntervalYearMonthWritable intervalYearMonth = ((HiveIntervalYearMonthObjectInspector) poi) + .getPrimitiveWritableObject(o); + return intervalYearMonth.hashCode(); + case INTERVAL_DAY_TIME: + HiveIntervalDayTimeWritable intervalDayTime = ((HiveIntervalDayTimeObjectInspector) poi) + .getPrimitiveWritableObject(o); + return intervalDayTime.hashCode(); case DECIMAL: return ((HiveDecimalObjectInspector) poi).getPrimitiveWritableObject(o).hashCode(); @@ -758,6 +772,20 @@ public final class ObjectInspectorUtils .getPrimitiveWritableObject(o2); return t1.compareTo(t2); } + case INTERVAL_YEAR_MONTH: { + HiveIntervalYearMonthWritable i1 = ((HiveIntervalYearMonthObjectInspector) poi1) + .getPrimitiveWritableObject(o1); + HiveIntervalYearMonthWritable i2 = ((HiveIntervalYearMonthObjectInspector) poi2) + .getPrimitiveWritableObject(o2); + return i1.compareTo(i2); + } + case INTERVAL_DAY_TIME: { + HiveIntervalDayTimeWritable i1 = ((HiveIntervalDayTimeObjectInspector) poi1) + .getPrimitiveWritableObject(o1); + HiveIntervalDayTimeWritable i2 = ((HiveIntervalDayTimeObjectInspector) poi2) + .getPrimitiveWritableObject(o2); + return i1.compareTo(i2); + } case DECIMAL: { HiveDecimalWritable t1 = ((HiveDecimalObjectInspector) poi1) .getPrimitiveWritableObject(o1); @@ -1092,6 +1120,10 @@ public final class ObjectInspectorUtils return oi instanceof SettableDateObjectInspector; case TIMESTAMP: return oi instanceof SettableTimestampObjectInspector; + case INTERVAL_YEAR_MONTH: + return oi instanceof SettableHiveIntervalYearMonthObjectInspector; + case INTERVAL_DAY_TIME: + return oi instanceof SettableHiveIntervalDayTimeObjectInspector; case BINARY: return oi instanceof SettableBinaryObjectInspector; case DECIMAL: Modified: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/PrimitiveObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/PrimitiveObjectInspector.java?rev=1669718&r1=1669717&r2=1669718&view=diff ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/PrimitiveObjectInspector.java (original) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/PrimitiveObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -31,7 +31,8 @@ public interface PrimitiveObjectInspecto */ public static enum PrimitiveCategory { VOID, BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, STRING, - DATE, TIMESTAMP, BINARY, DECIMAL, VARCHAR, CHAR, UNKNOWN + DATE, TIMESTAMP, BINARY, DECIMAL, VARCHAR, CHAR, INTERVAL_YEAR_MONTH, INTERVAL_DAY_TIME, + UNKNOWN }; public PrimitiveTypeInfo getTypeInfo(); Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/HiveIntervalDayTimeObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/HiveIntervalDayTimeObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/HiveIntervalDayTimeObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/HiveIntervalDayTimeObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -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.hadoop.hive.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; +import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; + +/** + * A HiveIntervalObjectInspector inspects an Object representing an Interval. + */ +public interface HiveIntervalDayTimeObjectInspector extends PrimitiveObjectInspector { + + HiveIntervalDayTimeWritable getPrimitiveWritableObject(Object o); + + HiveIntervalDayTime getPrimitiveJavaObject(Object o); +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/HiveIntervalYearMonthObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/HiveIntervalYearMonthObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/HiveIntervalYearMonthObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/HiveIntervalYearMonthObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -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.hadoop.hive.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; +import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; + +/** + * A HiveIntervalObjectInspector inspects an Object representing a year-month Interval. + */ +public interface HiveIntervalYearMonthObjectInspector extends PrimitiveObjectInspector { + + HiveIntervalYearMonthWritable getPrimitiveWritableObject(Object o); + + HiveIntervalYearMonth getPrimitiveJavaObject(Object o); +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantBinaryObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantBinaryObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantBinaryObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantBinaryObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,39 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; +import org.apache.hadoop.io.BytesWritable; + +public class JavaConstantBinaryObjectInspector extends + JavaBinaryObjectInspector implements ConstantObjectInspector { + private byte[] value; + + public JavaConstantBinaryObjectInspector(byte[] value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new BytesWritable(value); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantBooleanObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantBooleanObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantBooleanObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantBooleanObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,39 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; +import org.apache.hadoop.io.BooleanWritable; + +public class JavaConstantBooleanObjectInspector extends + JavaBooleanObjectInspector implements ConstantObjectInspector { + private Boolean value; + + public JavaConstantBooleanObjectInspector(Boolean value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new BooleanWritable(value); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantByteObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantByteObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantByteObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantByteObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,39 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.serde2.io.ByteWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; + +public class JavaConstantByteObjectInspector extends JavaByteObjectInspector + implements ConstantObjectInspector { + private Byte value; + + public JavaConstantByteObjectInspector(Byte value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new ByteWritable(value); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,41 @@ +/** + * 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.serde2.objectinspector.primitive; + +import java.sql.Date; + +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; + +public class JavaConstantDateObjectInspector extends JavaDateObjectInspector + implements ConstantObjectInspector { + private Date value; + + public JavaConstantDateObjectInspector(Date value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new DateWritable(value); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDoubleObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDoubleObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDoubleObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDoubleObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,39 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; + +public class JavaConstantDoubleObjectInspector extends + JavaDoubleObjectInspector implements ConstantObjectInspector { + private Double value; + + public JavaConstantDoubleObjectInspector(Double value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new DoubleWritable(value); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantFloatObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantFloatObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantFloatObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantFloatObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,39 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; +import org.apache.hadoop.io.FloatWritable; + +public class JavaConstantFloatObjectInspector extends JavaFloatObjectInspector + implements ConstantObjectInspector { + private Float value; + + public JavaConstantFloatObjectInspector(Float value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new FloatWritable(value); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantHiveCharObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantHiveCharObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantHiveCharObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantHiveCharObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,40 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.common.type.HiveChar; +import org.apache.hadoop.hive.serde2.io.HiveCharWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; + +public class JavaConstantHiveCharObjectInspector extends + JavaHiveCharObjectInspector implements ConstantObjectInspector { + private HiveChar value; + + public JavaConstantHiveCharObjectInspector(HiveChar value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new HiveCharWritable(value); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantHiveDecimalObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantHiveDecimalObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantHiveDecimalObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantHiveDecimalObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,40 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; + +public class JavaConstantHiveDecimalObjectInspector extends + JavaHiveDecimalObjectInspector implements ConstantObjectInspector { + private HiveDecimal value; + + public JavaConstantHiveDecimalObjectInspector(HiveDecimal value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new HiveDecimalWritable(value); + } +} \ No newline at end of file Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantHiveVarcharObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantHiveVarcharObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantHiveVarcharObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantHiveVarcharObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,40 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; + +public class JavaConstantHiveVarcharObjectInspector extends + JavaHiveVarcharObjectInspector implements ConstantObjectInspector { + private HiveVarchar value; + + public JavaConstantHiveVarcharObjectInspector(HiveVarchar value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new HiveVarcharWritable(value); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantIntObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantIntObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantIntObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantIntObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,39 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; +import org.apache.hadoop.io.IntWritable; + +public class JavaConstantIntObjectInspector extends JavaIntObjectInspector + implements ConstantObjectInspector { + private Integer value; + + public JavaConstantIntObjectInspector(Integer value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new IntWritable(value); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantLongObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantLongObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantLongObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantLongObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,39 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; +import org.apache.hadoop.io.LongWritable; + +public class JavaConstantLongObjectInspector extends JavaLongObjectInspector + implements ConstantObjectInspector { + private Long value; + + public JavaConstantLongObjectInspector(Long value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new LongWritable(value); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantShortObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantShortObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantShortObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantShortObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,39 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.serde2.io.ShortWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; + +public class JavaConstantShortObjectInspector extends JavaDateObjectInspector + implements ConstantObjectInspector { + private Short value; + + public JavaConstantShortObjectInspector(Short value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new ShortWritable(value); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantStringObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantStringObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantStringObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantStringObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,39 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; +import org.apache.hadoop.io.Text; + +public class JavaConstantStringObjectInspector extends + JavaStringObjectInspector implements ConstantObjectInspector { + private String value; + + public JavaConstantStringObjectInspector(String value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new Text(value); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,41 @@ +/** + * 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.serde2.objectinspector.primitive; + +import java.sql.Timestamp; + +import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; + +public class JavaConstantTimestampObjectInspector extends + JavaTimestampObjectInspector implements ConstantObjectInspector { + private Timestamp value; + + public JavaConstantTimestampObjectInspector(Timestamp value) { + super(); + this.value = value; + } + + @Override + public Object getWritableConstantValue() { + if (value==null) { + return null; + } + return new TimestampWritable(value); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaHiveIntervalDayTimeObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaHiveIntervalDayTimeObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaHiveIntervalDayTimeObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaHiveIntervalDayTimeObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,57 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; +import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; + +public class JavaHiveIntervalDayTimeObjectInspector + extends AbstractPrimitiveJavaObjectInspector + implements SettableHiveIntervalDayTimeObjectInspector{ + + public JavaHiveIntervalDayTimeObjectInspector() { + super(TypeInfoFactory.intervalDayTimeTypeInfo); + } + + @Override + public HiveIntervalDayTime getPrimitiveJavaObject(Object o) { + return o == null ? null : (HiveIntervalDayTime) o; + } + + @Override + public HiveIntervalDayTimeWritable getPrimitiveWritableObject(Object o) { + return o == null ? null : new HiveIntervalDayTimeWritable((HiveIntervalDayTime) o); + } + + @Override + public Object set(Object o, HiveIntervalDayTime i) { + return i == null ? null : new HiveIntervalDayTime(i); + } + + @Override + public Object set(Object o, HiveIntervalDayTimeWritable i) { + return i == null ? null : i.getHiveIntervalDayTime(); + } + + @Override + public Object create(HiveIntervalDayTime i) { + return i == null ? null : new HiveIntervalDayTime(i); + } +} Added: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaHiveIntervalYearMonthObjectInspector.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaHiveIntervalYearMonthObjectInspector.java?rev=1669718&view=auto ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaHiveIntervalYearMonthObjectInspector.java (added) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaHiveIntervalYearMonthObjectInspector.java Sat Mar 28 00:22:15 2015 @@ -0,0 +1,57 @@ +/** + * 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.serde2.objectinspector.primitive; + +import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; +import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; + +public class JavaHiveIntervalYearMonthObjectInspector + extends AbstractPrimitiveJavaObjectInspector + implements SettableHiveIntervalYearMonthObjectInspector{ + + public JavaHiveIntervalYearMonthObjectInspector() { + super(TypeInfoFactory.intervalYearMonthTypeInfo); + } + + @Override + public HiveIntervalYearMonth getPrimitiveJavaObject(Object o) { + return o == null ? null : (HiveIntervalYearMonth) o; + } + + @Override + public HiveIntervalYearMonthWritable getPrimitiveWritableObject(Object o) { + return o == null ? null : new HiveIntervalYearMonthWritable((HiveIntervalYearMonth) o); + } + + @Override + public Object set(Object o, HiveIntervalYearMonth i) { + return i == null ? null : new HiveIntervalYearMonth(i); + } + + @Override + public Object set(Object o, HiveIntervalYearMonthWritable i) { + return i == null ? null : i.getHiveIntervalYearMonth(); + } + + @Override + public Object create(HiveIntervalYearMonth i) { + return i == null ? null : new HiveIntervalYearMonth(i); + } +} Modified: hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java URL: http://svn.apache.org/viewvc/hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java?rev=1669718&r1=1669717&r2=1669718&view=diff ============================================================================== --- hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java (original) +++ hive/branches/llap/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java Sat Mar 28 00:22:15 2015 @@ -23,6 +23,8 @@ import java.sql.Timestamp; import org.apache.hadoop.hive.common.type.HiveChar; import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; +import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; import org.apache.hadoop.hive.common.type.HiveVarchar; import org.apache.hadoop.hive.serde2.ByteStream; import org.apache.hadoop.hive.serde2.io.HiveCharWritable; @@ -285,6 +287,46 @@ public class PrimitiveObjectInspectorCon } } + public static class HiveIntervalYearMonthConverter implements Converter { + PrimitiveObjectInspector inputOI; + SettableHiveIntervalYearMonthObjectInspector outputOI; + Object r; + + public HiveIntervalYearMonthConverter(PrimitiveObjectInspector inputOI, + SettableHiveIntervalYearMonthObjectInspector outputOI) { + this.inputOI = inputOI; + this.outputOI = outputOI; + r = outputOI.create(new HiveIntervalYearMonth()); + } + + public Object convert(Object input) { + if (input == null) { + return null; + } + return outputOI.set(r, PrimitiveObjectInspectorUtils.getHiveIntervalYearMonth(input, inputOI)); + } + } + + public static class HiveIntervalDayTimeConverter implements Converter { + PrimitiveObjectInspector inputOI; + SettableHiveIntervalDayTimeObjectInspector outputOI; + Object r; + + public HiveIntervalDayTimeConverter(PrimitiveObjectInspector inputOI, + SettableHiveIntervalDayTimeObjectInspector outputOI) { + this.inputOI = inputOI; + this.outputOI = outputOI; + r = outputOI.create(new HiveIntervalDayTime()); + } + + public Object convert(Object input) { + if (input == null) { + return null; + } + return outputOI.set(r, PrimitiveObjectInspectorUtils.getHiveIntervalDayTime(input, inputOI)); + } + } + public static class HiveDecimalConverter implements Converter { PrimitiveObjectInspector inputOI; @@ -419,6 +461,14 @@ public class PrimitiveObjectInspectorCon t.set(((TimestampObjectInspector) inputOI) .getPrimitiveWritableObject(input).toString()); return t; + case INTERVAL_YEAR_MONTH: + t.set(((HiveIntervalYearMonthObjectInspector) inputOI) + .getPrimitiveWritableObject(input).toString()); + return t; + case INTERVAL_DAY_TIME: + t.set(((HiveIntervalDayTimeObjectInspector) inputOI) + .getPrimitiveWritableObject(input).toString()); + return t; case BINARY: BinaryObjectInspector binaryOI = (BinaryObjectInspector) inputOI; if (binaryOI.preferWritable()) {