http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritableV2.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritableV2.java b/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritableV2.java new file mode 100644 index 0000000..9aa7f19 --- /dev/null +++ b/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritableV2.java @@ -0,0 +1,625 @@ +/* + * 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.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.time.format.DateTimeFormatter; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.common.type.Timestamp; +import org.apache.hadoop.hive.common.type.TimestampUtils; +import org.apache.hadoop.hive.serde2.ByteStream.RandomAccessOutput; +import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils; +import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VInt; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.io.WritableUtils; + +/** + * TimestampWritableV2 + * + * Timestamps are of the format + * YYYY-MM-DD HH:MM:SS.[fff...] + * + * We encode Unix timestamp in seconds in 4 bytes, using the MSB to signify + * whether the timestamp has a fractional portion. + * + * The fractional portion is reversed, and encoded as a VInt + * so timestamps with less precision use fewer bytes. + * + * 0.1 -> 1 + * 0.01 -> 10 + * 0.001 -> 100 + * + */ +public class TimestampWritableV2 implements WritableComparable<TimestampWritableV2> { + + static final public byte[] nullBytes = {0x0, 0x0, 0x0, 0x0}; + + private static final int DECIMAL_OR_SECOND_VINT_FLAG = 0x80000000; + private static final int LOWEST_31_BITS_OF_SEC_MASK = 0x7fffffff; + + private static final long SEVEN_BYTE_LONG_SIGN_FLIP = 0xff80L << 48; + + + /** The maximum number of bytes required for a TimestampWritableV2 */ + public static final int MAX_BYTES = 13; + + public static final int BINARY_SORTABLE_LENGTH = 11; + + public static final DateTimeFormatter DATE_TIME_FORMAT = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + + private Timestamp timestamp = new Timestamp(); + + /** + * true if data is stored in timestamp field rather than byte arrays. + * allows for lazy conversion to bytes when necessary + * false otherwise + */ + private boolean bytesEmpty; + private boolean timestampEmpty; + + /* Allow use of external byte[] for efficiency */ + private byte[] currentBytes; + private final byte[] internalBytes = new byte[MAX_BYTES]; + private byte[] externalBytes; + private int offset; + + /* Constructors */ + public TimestampWritableV2() { + bytesEmpty = false; + currentBytes = internalBytes; + offset = 0; + + clearTimestamp(); + } + + public TimestampWritableV2(byte[] bytes, int offset) { + set(bytes, offset); + } + + public TimestampWritableV2(TimestampWritableV2 t) { + this(t.getBytes(), 0); + } + + public TimestampWritableV2(Timestamp t) { + set(t); + } + + public void set(byte[] bytes, int offset) { + externalBytes = bytes; + this.offset = offset; + bytesEmpty = false; + currentBytes = externalBytes; + + clearTimestamp(); + } + + public void set(Timestamp t) { + if (t == null) { + timestamp.set(null); + return; + } + timestamp.set(t); + bytesEmpty = true; + timestampEmpty = false; + } + + public void set(TimestampWritableV2 t) { + if (t.bytesEmpty) { + set(t.getTimestamp()); + return; + } + if (t.currentBytes == t.externalBytes) { + set(t.currentBytes, t.offset); + } else { + set(t.currentBytes, 0); + } + } + + public static void updateTimestamp(Timestamp timestamp, long secondsAsMillis, int nanos) { + timestamp.setTimeInMillis(secondsAsMillis, nanos); + } + + public void setInternal(long secondsAsMillis, int nanos) { + // This is our way of documenting that we are MUTATING the contents of + // this writable's internal timestamp. + updateTimestamp(timestamp, secondsAsMillis, nanos); + + bytesEmpty = true; + timestampEmpty = false; + } + + private void clearTimestamp() { + timestampEmpty = true; + } + + public void writeToByteStream(RandomAccessOutput byteStream) { + checkBytes(); + byteStream.write(currentBytes, offset, getTotalLength()); + } + + /** + * + * @return seconds corresponding to this TimestampWritableV2 + */ + public long getSeconds() { + if (!timestampEmpty) { + return timestamp.toEpochSecond(); + } else if (!bytesEmpty) { + return TimestampWritableV2.getSeconds(currentBytes, offset); + } else { + throw new IllegalStateException("Both timestamp and bytes are empty"); + } + } + + /** + * + * @return nanoseconds in this TimestampWritableV2 + */ + public int getNanos() { + if (!timestampEmpty) { + return timestamp.getNanos(); + } else if (!bytesEmpty) { + return hasDecimalOrSecondVInt() ? + TimestampWritableV2.getNanos(currentBytes, offset + 4) : 0; + } else { + throw new IllegalStateException("Both timestamp and bytes are empty"); + } + } + + /** + * @return length of serialized TimestampWritableV2 data. As a side effect, populates the internal + * byte array if empty. + */ + int getTotalLength() { + checkBytes(); + return getTotalLength(currentBytes, offset); + } + + public static int getTotalLength(byte[] bytes, int offset) { + int len = 4; + if (hasDecimalOrSecondVInt(bytes[offset])) { + int firstVIntLen = WritableUtils.decodeVIntSize(bytes[offset + 4]); + len += firstVIntLen; + if (hasSecondVInt(bytes[offset + 4])) { + len += WritableUtils.decodeVIntSize(bytes[offset + 4 + firstVIntLen]); + } + } + return len; + } + + public Timestamp getTimestamp() { + if (timestampEmpty) { + populateTimestamp(); + } + return timestamp; + } + + /** + * Used to create copies of objects + * @return a copy of the internal TimestampWritableV2 byte[] + */ + public byte[] getBytes() { + checkBytes(); + + int len = getTotalLength(); + byte[] b = new byte[len]; + + System.arraycopy(currentBytes, offset, b, 0, len); + return b; + } + + /** + * @return byte[] representation of TimestampWritableV2 that is binary + * sortable (7 bytes for seconds, 4 bytes for nanoseconds) + */ + public byte[] getBinarySortable() { + byte[] b = new byte[BINARY_SORTABLE_LENGTH]; + int nanos = getNanos(); + // We flip the highest-order bit of the seven-byte representation of seconds to make negative + // values come before positive ones. + long seconds = getSeconds() ^ SEVEN_BYTE_LONG_SIGN_FLIP; + sevenByteLongToBytes(seconds, b, 0); + intToBytes(nanos, b, 7); + return b; + } + + /** + * Given a byte[] that has binary sortable data, initialize the internal + * structures to hold that data + * @param bytes the byte array that holds the binary sortable representation + * @param binSortOffset offset of the binary-sortable representation within the buffer. + */ + public void setBinarySortable(byte[] bytes, int binSortOffset) { + // Flip the sign bit (and unused bits of the high-order byte) of the seven-byte long back. + long seconds = readSevenByteLong(bytes, binSortOffset) ^ SEVEN_BYTE_LONG_SIGN_FLIP; + int nanos = bytesToInt(bytes, binSortOffset + 7); + int firstInt = (int) seconds; + boolean hasSecondVInt = seconds < 0 || seconds > Integer.MAX_VALUE; + if (nanos != 0 || hasSecondVInt) { + firstInt |= DECIMAL_OR_SECOND_VINT_FLAG; + } else { + firstInt &= LOWEST_31_BITS_OF_SEC_MASK; + } + + intToBytes(firstInt, internalBytes, 0); + setNanosBytes(nanos, internalBytes, 4, hasSecondVInt); + if (hasSecondVInt) { + LazyBinaryUtils.writeVLongToByteArray(internalBytes, + 4 + WritableUtils.decodeVIntSize(internalBytes[4]), + seconds >> 31); + } + + currentBytes = internalBytes; + this.offset = 0; + } + + /** + * The data of TimestampWritableV2 can be stored either in a byte[] + * or in a Timestamp object. Calling this method ensures that the byte[] + * is populated from the Timestamp object if previously empty. + */ + private void checkBytes() { + if (bytesEmpty) { + // Populate byte[] from Timestamp + convertTimestampToBytes(timestamp, internalBytes, 0); + offset = 0; + currentBytes = internalBytes; + bytesEmpty = false; + } + } + + /** + * + * @return double representation of the timestamp, accurate to nanoseconds + */ + public double getDouble() { + double seconds, nanos; + if (bytesEmpty) { + seconds = timestamp.toEpochSecond(); + nanos = timestamp.getNanos(); + } else { + seconds = getSeconds(); + nanos = getNanos(); + } + return seconds + nanos / 1000000000; + } + + public static long getLong(Timestamp timestamp) { + return timestamp.toEpochSecond(); + } + + public void readFields(DataInput in) throws IOException { + in.readFully(internalBytes, 0, 4); + if (TimestampWritableV2.hasDecimalOrSecondVInt(internalBytes[0])) { + in.readFully(internalBytes, 4, 1); + int len = (byte) WritableUtils.decodeVIntSize(internalBytes[4]); + if (len > 1) { + in.readFully(internalBytes, 5, len-1); + } + + long vlong = LazyBinaryUtils.readVLongFromByteArray(internalBytes, 4); + if (vlong < -1000000000 || vlong > 999999999) { + throw new IOException( + "Invalid first vint value (encoded nanoseconds) of a TimestampWritableV2: " + vlong + + ", expected to be between -1000000000 and 999999999."); + // Note that -1000000000 is a valid value corresponding to a nanosecond timestamp + // of 999999999, because if the second VInt is present, we use the value + // (-reversedNanoseconds - 1) as the second VInt. + } + if (vlong < 0) { + // This indicates there is a second VInt containing the additional bits of the seconds + // field. + in.readFully(internalBytes, 4 + len, 1); + int secondVIntLen = (byte) WritableUtils.decodeVIntSize(internalBytes[4 + len]); + if (secondVIntLen > 1) { + in.readFully(internalBytes, 5 + len, secondVIntLen - 1); + } + } + } + currentBytes = internalBytes; + this.offset = 0; + } + + public void write(DataOutput out) throws IOException { + checkBytes(); + out.write(currentBytes, offset, getTotalLength()); + } + + public int compareTo(TimestampWritableV2 t) { + checkBytes(); + long s1 = this.getSeconds(); + long s2 = t.getSeconds(); + if (s1 == s2) { + int n1 = this.getNanos(); + int n2 = t.getNanos(); + if (n1 == n2) { + return 0; + } + return n1 - n2; + } else { + return s1 < s2 ? -1 : 1; + } + } + + @Override + public boolean equals(Object o) { + return compareTo((TimestampWritableV2) o) == 0; + } + + @Override + public String toString() { + if (timestampEmpty) { + populateTimestamp(); + } + + if (timestamp.getNanos() > 0) { + return timestamp.toString(); + } + + String timestampString = timestamp.toString(); + if (timestampString.length() > 19) { + if (timestampString.length() == 21) { + if (timestampString.substring(19).compareTo(".0") == 0) { + return timestamp.format(DATE_TIME_FORMAT); + } + } + return timestamp.format(DATE_TIME_FORMAT) + timestampString.substring(19); + } + + return timestamp.format(DATE_TIME_FORMAT); + } + + @Override + public int hashCode() { + long seconds = getSeconds(); + seconds <<= 30; // the nanosecond part fits in 30 bits + seconds |= getNanos(); + return (int) ((seconds >>> 32) ^ seconds); + } + + private void populateTimestamp() { + long seconds = getSeconds(); + int nanos = getNanos(); + timestamp.setTimeInSeconds(seconds, nanos); + } + + /** Static methods **/ + + /** + * Gets seconds stored as integer at bytes[offset] + * @param bytes + * @param offset + * @return the number of seconds + */ + public static long getSeconds(byte[] bytes, int offset) { + int lowest31BitsOfSecondsAndFlag = bytesToInt(bytes, offset); + if (lowest31BitsOfSecondsAndFlag >= 0 || // the "has decimal or second VInt" flag is not set + !hasSecondVInt(bytes[offset + 4])) { + // The entire seconds field is stored in the first 4 bytes. + return lowest31BitsOfSecondsAndFlag & LOWEST_31_BITS_OF_SEC_MASK; + } + + // We compose the seconds field from two parts. The lowest 31 bits come from the first four + // bytes. The higher-order bits come from the second VInt that follows the nanos field. + return ((long) (lowest31BitsOfSecondsAndFlag & LOWEST_31_BITS_OF_SEC_MASK)) | + (LazyBinaryUtils.readVLongFromByteArray(bytes, + offset + 4 + WritableUtils.decodeVIntSize(bytes[offset + 4])) << 31); + } + + public static int getNanos(byte[] bytes, int offset) { + VInt vInt = LazyBinaryUtils.threadLocalVInt.get(); + LazyBinaryUtils.readVInt(bytes, offset, vInt); + int val = vInt.value; + if (val < 0) { + // This means there is a second VInt present that specifies additional bits of the timestamp. + // The reversed nanoseconds value is still encoded in this VInt. + val = -val - 1; + } + int len = (int) Math.floor(Math.log10(val)) + 1; + + // Reverse the value + int tmp = 0; + while (val != 0) { + tmp *= 10; + tmp += val % 10; + val /= 10; + } + val = tmp; + + if (len < 9) { + val *= Math.pow(10, 9 - len); + } + return val; + } + + /** + * Writes a Timestamp's serialized value to byte array b at the given offset + * @param t to convert to bytes + * @param b destination byte array + * @param offset destination offset in the byte array + */ + public static void convertTimestampToBytes(Timestamp t, byte[] b, + int offset) { + long seconds = t.toEpochSecond(); + int nanos = t.getNanos(); + + boolean hasSecondVInt = seconds < 0 || seconds > Integer.MAX_VALUE; + boolean hasDecimal = setNanosBytes(nanos, b, offset+4, hasSecondVInt); + + int firstInt = (int) seconds; + if (hasDecimal || hasSecondVInt) { + firstInt |= DECIMAL_OR_SECOND_VINT_FLAG; + } else { + firstInt &= LOWEST_31_BITS_OF_SEC_MASK; + } + intToBytes(firstInt, b, offset); + + if (hasSecondVInt) { + LazyBinaryUtils.writeVLongToByteArray(b, + offset + 4 + WritableUtils.decodeVIntSize(b[offset + 4]), + seconds >> 31); + } + } + + /** + * Given an integer representing nanoseconds, write its serialized + * value to the byte array b at offset + * + * @param nanos + * @param b + * @param offset + * @return + */ + private static boolean setNanosBytes(int nanos, byte[] b, int offset, boolean hasSecondVInt) { + int decimal = 0; + if (nanos != 0) { + int counter = 0; + while (counter < 9) { + decimal *= 10; + decimal += nanos % 10; + nanos /= 10; + counter++; + } + } + + if (hasSecondVInt || decimal != 0) { + // We use the sign of the reversed-nanoseconds field to indicate that there is a second VInt + // present. + LazyBinaryUtils.writeVLongToByteArray(b, offset, hasSecondVInt ? (-decimal - 1) : decimal); + } + return decimal != 0; + } + + public HiveDecimal getHiveDecimal() { + if (timestampEmpty) { + populateTimestamp(); + } + return getHiveDecimal(timestamp); + } + + public static HiveDecimal getHiveDecimal(Timestamp timestamp) { + // The BigDecimal class recommends not converting directly from double to BigDecimal, + // so we convert through a string... + Double timestampDouble = TimestampUtils.getDouble(timestamp); + HiveDecimal result = HiveDecimal.create(timestampDouble.toString()); + return result; + } + + /** + * Converts the time in seconds or milliseconds to a timestamp. + * @param time time in seconds or in milliseconds + * @return the timestamp + */ + public static Timestamp longToTimestamp(long time, boolean intToTimestampInSeconds) { + // If the time is in seconds, converts it to milliseconds first. + if (intToTimestampInSeconds) { + return Timestamp.ofEpochSecond(time); + } + return Timestamp.ofEpochMilli(time); + } + + public static void setTimestamp(Timestamp t, byte[] bytes, int offset) { + long seconds = getSeconds(bytes, offset); + int nanos; + if (hasDecimalOrSecondVInt(bytes[offset])) { + nanos = getNanos(bytes, offset + 4); + } else { + nanos = 0; + } + t.setTimeInSeconds(seconds, nanos); + } + + public static Timestamp createTimestamp(byte[] bytes, int offset) { + Timestamp t = new Timestamp(); + TimestampWritableV2.setTimestamp(t, bytes, offset); + return t; + } + + private static boolean hasDecimalOrSecondVInt(byte b) { + return (b >> 7) != 0; + } + + private static boolean hasSecondVInt(byte b) { + return WritableUtils.isNegativeVInt(b); + } + + private final boolean hasDecimalOrSecondVInt() { + return hasDecimalOrSecondVInt(currentBytes[offset]); + } + + public final boolean hasDecimal() { + return hasDecimalOrSecondVInt() || currentBytes[offset + 4] != -1; + // If the first byte of the VInt is -1, the VInt itself is -1, indicating that there is a + // second VInt but the nanoseconds field is actually 0. + } + + /** + * Writes <code>value</code> into <code>dest</code> at <code>offset</code> + * @param value + * @param dest + * @param offset + */ + private static void intToBytes(int value, byte[] dest, int offset) { + dest[offset] = (byte) ((value >> 24) & 0xFF); + dest[offset+1] = (byte) ((value >> 16) & 0xFF); + dest[offset+2] = (byte) ((value >> 8) & 0xFF); + dest[offset+3] = (byte) (value & 0xFF); + } + + /** + * Writes <code>value</code> into <code>dest</code> at <code>offset</code> as a seven-byte + * serialized long number. + */ + static void sevenByteLongToBytes(long value, byte[] dest, int offset) { + dest[offset] = (byte) ((value >> 48) & 0xFF); + dest[offset+1] = (byte) ((value >> 40) & 0xFF); + dest[offset+2] = (byte) ((value >> 32) & 0xFF); + dest[offset+3] = (byte) ((value >> 24) & 0xFF); + dest[offset+4] = (byte) ((value >> 16) & 0xFF); + dest[offset+5] = (byte) ((value >> 8) & 0xFF); + dest[offset+6] = (byte) (value & 0xFF); + } + + /** + * + * @param bytes + * @param offset + * @return integer represented by the four bytes in <code>bytes</code> + * beginning at <code>offset</code> + */ + private static int bytesToInt(byte[] bytes, int offset) { + return ((0xFF & bytes[offset]) << 24) + | ((0xFF & bytes[offset+1]) << 16) + | ((0xFF & bytes[offset+2]) << 8) + | (0xFF & bytes[offset+3]); + } + + static long readSevenByteLong(byte[] bytes, int offset) { + // We need to shift everything 8 bits left and then shift back to populate the sign field. + return (((0xFFL & bytes[offset]) << 56) + | ((0xFFL & bytes[offset+1]) << 48) + | ((0xFFL & bytes[offset+2]) << 40) + | ((0xFFL & bytes[offset+3]) << 32) + | ((0xFFL & bytes[offset+4]) << 24) + | ((0xFFL & bytes[offset+5]) << 16) + | ((0xFFL & bytes[offset+6]) << 8)) >> 8; + } +}
http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDate.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDate.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDate.java index c50cd40..e464339 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDate.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDate.java @@ -20,11 +20,11 @@ package org.apache.hadoop.hive.serde2.lazy; import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; -import java.sql.Date; +import org.apache.hadoop.hive.common.type.Date; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.serde2.io.DateWritable; import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyDateObjectInspector; import org.apache.hadoop.io.Text; @@ -36,17 +36,17 @@ import org.apache.hadoop.io.Text; * YYYY-MM-DD * */ -public class LazyDate extends LazyPrimitive<LazyDateObjectInspector, DateWritable> { +public class LazyDate extends LazyPrimitive<LazyDateObjectInspector, DateWritableV2> { private static final Logger LOG = LoggerFactory.getLogger(LazyDate.class); public LazyDate(LazyDateObjectInspector oi) { super(oi); - data = new DateWritable(); + data = new DateWritableV2(); } public LazyDate(LazyDate copy) { super(copy); - data = new DateWritable(copy.data); + data = new DateWritableV2(copy.data); } /** @@ -81,7 +81,7 @@ public class LazyDate extends LazyPrimitive<LazyDateObjectInspector, DateWritabl * The Date to write * @throws IOException */ - public static void writeUTF8(OutputStream out, DateWritable d) + public static void writeUTF8(OutputStream out, DateWritableV2 d) throws IOException { ByteBuffer b = Text.encode(d.toString()); out.write(b.array(), 0, b.limit()); http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestamp.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestamp.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestamp.java index ee801ee..3473c56 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestamp.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestamp.java @@ -20,11 +20,11 @@ package org.apache.hadoop.hive.serde2.lazy; import java.io.IOException; import java.io.OutputStream; import java.io.UnsupportedEncodingException; -import java.sql.Timestamp; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.common.type.Timestamp; import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyTimestampObjectInspector; /** @@ -35,17 +35,17 @@ import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyTimestam * YYYY-MM-DD HH:MM:SS.[fff...] * */ -public class LazyTimestamp extends LazyPrimitive<LazyTimestampObjectInspector, TimestampWritable> { +public class LazyTimestamp extends LazyPrimitive<LazyTimestampObjectInspector, TimestampWritableV2> { private static final Logger LOG = LoggerFactory.getLogger(LazyTimestamp.class); public LazyTimestamp(LazyTimestampObjectInspector oi) { super(oi); - data = new TimestampWritable(); + data = new TimestampWritableV2(); } public LazyTimestamp(LazyTimestamp copy) { super(copy); - data = new TimestampWritable(copy.data); + data = new TimestampWritableV2(copy.data); } /** @@ -94,18 +94,18 @@ public class LazyTimestamp extends LazyPrimitive<LazyTimestampObjectInspector, T * The Timestamp to write * @throws IOException */ - public static void writeUTF8(OutputStream out, TimestampWritable i) + public static void writeUTF8(OutputStream out, TimestampWritableV2 i) throws IOException { if (i == null) { // Serialize as time 0 - out.write(TimestampWritable.nullBytes); + out.write(TimestampWritableV2.nullBytes); } else { out.write(i.toString().getBytes("US-ASCII")); } } @Override - public TimestampWritable getWritableObject() { + public TimestampWritableV2 getWritableObject() { return data; } } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazy/VerifyLazy.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/VerifyLazy.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/VerifyLazy.java index 17c0357..14ff6d2 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/VerifyLazy.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/VerifyLazy.java @@ -17,20 +17,20 @@ */ package org.apache.hadoop.hive.serde2.lazy; -import java.sql.Date; -import java.sql.Timestamp; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; +import org.apache.hadoop.hive.common.type.Date; import org.apache.hadoop.hive.common.type.HiveChar; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hadoop.hive.common.type.Timestamp; import org.apache.hadoop.hive.serde2.io.ByteWritable; -import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; import org.apache.hadoop.hive.serde2.io.DoubleWritable; import org.apache.hadoop.hive.serde2.io.HiveCharWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; @@ -38,7 +38,7 @@ 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.ShortWritable; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryArray; import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryMap; import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryStruct; @@ -301,7 +301,7 @@ public class VerifyLazy { throw new RuntimeException("Expected LazyDate"); } Date value = ((LazyDate) primitiveObject).getWritableObject().get(); - Date expected = ((DateWritable) expectedObject).get(); + Date expected = ((DateWritableV2) expectedObject).get(); if (!value.equals(expected)) { throw new RuntimeException("Date field mismatch (expected " + expected + " found " + value + ")"); } @@ -310,10 +310,10 @@ public class VerifyLazy { case TIMESTAMP: { if (!(primitiveObject instanceof LazyTimestamp)) { - throw new RuntimeException("TimestampWritable expected writable not TimestampWritable"); + throw new RuntimeException("TimestampWritableV2 expected writable not TimestampWritableV2"); } Timestamp value = ((LazyTimestamp) primitiveObject).getWritableObject().getTimestamp(); - Timestamp expected = ((TimestampWritable) expectedObject).getTimestamp(); + Timestamp expected = ((TimestampWritableV2) expectedObject).getTimestamp(); if (!value.equals(expected)) { throw new RuntimeException("Timestamp field mismatch (expected " + expected + " found " + value + ")"); } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java index 1890f18..45c44da 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java @@ -21,10 +21,10 @@ package org.apache.hadoop.hive.serde2.lazy.fast; import java.io.IOException; import java.nio.charset.CharacterCodingException; import java.nio.charset.StandardCharsets; -import java.sql.Date; import java.util.Arrays; import java.util.List; +import org.apache.hadoop.hive.common.type.Date; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation; http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java index 356326c..a42d6f4 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java @@ -20,14 +20,14 @@ package org.apache.hadoop.hive.serde2.lazy.fast; import java.io.IOException; import java.nio.ByteBuffer; -import java.sql.Date; -import java.sql.Timestamp; import java.util.ArrayDeque; import java.util.Deque; import java.util.List; import java.util.Map; import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.hive.common.type.Date; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.common.type.HiveChar; @@ -35,12 +35,12 @@ import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hadoop.hive.common.type.Timestamp; import org.apache.hadoop.hive.serde2.ByteStream.Output; -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.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.lazy.LazyDate; import org.apache.hadoop.hive.serde2.lazy.LazyHiveDecimal; import org.apache.hadoop.hive.serde2.lazy.LazyHiveIntervalDayTime; @@ -77,8 +77,8 @@ public final class LazySimpleSerializeWrite implements SerializeWrite { private Deque<Integer> indexStack = new ArrayDeque<Integer>(); // For thread safety, we allocate private writable objects for our use only. - private DateWritable dateWritable; - private TimestampWritable timestampWritable; + private DateWritableV2 dateWritable; + private TimestampWritableV2 timestampWritable; private HiveIntervalYearMonthWritable hiveIntervalYearMonthWritable; private HiveIntervalDayTimeWritable hiveIntervalDayTimeWritable; private HiveIntervalDayTime hiveIntervalDayTime; @@ -299,7 +299,7 @@ public final class LazySimpleSerializeWrite implements SerializeWrite { public void writeDate(Date date) throws IOException { beginPrimitive(); if (dateWritable == null) { - dateWritable = new DateWritable(); + dateWritable = new DateWritableV2(); } dateWritable.set(date); LazyDate.writeUTF8(output, dateWritable); @@ -311,7 +311,7 @@ public final class LazySimpleSerializeWrite implements SerializeWrite { public void writeDate(int dateAsDays) throws IOException { beginPrimitive(); if (dateWritable == null) { - dateWritable = new DateWritable(); + dateWritable = new DateWritableV2(); } dateWritable.set(dateAsDays); LazyDate.writeUTF8(output, dateWritable); @@ -325,7 +325,7 @@ public final class LazySimpleSerializeWrite implements SerializeWrite { public void writeTimestamp(Timestamp v) throws IOException { beginPrimitive(); if (timestampWritable == null) { - timestampWritable = new TimestampWritable(); + timestampWritable = new TimestampWritableV2(); } timestampWritable.set(v); LazyTimestamp.writeUTF8(output, timestampWritable); http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyDateObjectInspector.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyDateObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyDateObjectInspector.java index 3bc4ff7..e356d23 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyDateObjectInspector.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyDateObjectInspector.java @@ -17,18 +17,17 @@ */ package org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive; -import java.sql.Date; - -import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.common.type.Date; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; import org.apache.hadoop.hive.serde2.lazy.LazyDate; import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; /** - * A WritableDateObjectInspector inspects a DateWritable Object. + * A WritableDateObjectInspector inspects a DateWritableV2 Object. */ public class LazyDateObjectInspector - extends AbstractPrimitiveLazyObjectInspector<DateWritable> + extends AbstractPrimitiveLazyObjectInspector<DateWritableV2> implements DateObjectInspector { protected LazyDateObjectInspector() { http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampObjectInspector.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampObjectInspector.java index e0f993e..a10a722 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampObjectInspector.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampObjectInspector.java @@ -17,17 +17,17 @@ */ package org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive; -import java.sql.Timestamp; import java.util.List; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.common.type.Timestamp; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.lazy.LazyTimestamp; import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hive.common.util.TimestampParser; public class LazyTimestampObjectInspector - extends AbstractPrimitiveLazyObjectInspector<TimestampWritable> + extends AbstractPrimitiveLazyObjectInspector<TimestampWritableV2> implements TimestampObjectInspector { protected List<String> timestampFormats = null; http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java index 2952e26..fbfe961 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.hive.serde2.lazybinary; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.serde2.io.DateWritable; 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.WritableDateObjectInspector; @@ -29,17 +29,17 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableDateObjec * */ public class LazyBinaryDate extends - LazyBinaryPrimitive<WritableDateObjectInspector, DateWritable> { + LazyBinaryPrimitive<WritableDateObjectInspector, DateWritableV2> { static final Logger LOG = LoggerFactory.getLogger(LazyBinaryDate.class); LazyBinaryDate(WritableDateObjectInspector oi) { super(oi); - data = new DateWritable(); + data = new DateWritableV2(); } LazyBinaryDate(LazyBinaryDate copy) { super(copy); - data = new DateWritable(copy.data); + data = new DateWritableV2(copy.data); } /** http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java index 24704a1..660080c 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable; import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampLocalTZObjectInspector; import org.slf4j.Logger; @@ -37,12 +38,11 @@ import org.apache.hadoop.hive.serde2.SerDeException; import org.apache.hadoop.hive.serde2.SerDeSpec; import org.apache.hadoop.hive.serde2.SerDeStats; import org.apache.hadoop.hive.serde2.SerDeUtils; -import org.apache.hadoop.hive.serde2.io.DateWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; import org.apache.hadoop.hive.common.type.HiveDecimal; 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.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef; import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector; @@ -318,7 +318,7 @@ public class LazyBinarySerDe extends AbstractSerDe { } public static void writeDateToByteStream(RandomAccessOutput byteStream, - DateWritable date) { + DateWritableV2 date) { LazyBinaryUtils.writeVInt(byteStream, date.getDays()); } @@ -505,13 +505,13 @@ public class LazyBinarySerDe extends AbstractSerDe { } case DATE: { - DateWritable d = ((DateObjectInspector) poi).getPrimitiveWritableObject(obj); + DateWritableV2 d = ((DateObjectInspector) poi).getPrimitiveWritableObject(obj); writeDateToByteStream(byteStream, d); return; } case TIMESTAMP: { TimestampObjectInspector toi = (TimestampObjectInspector) poi; - TimestampWritable t = toi.getPrimitiveWritableObject(obj); + TimestampWritableV2 t = toi.getPrimitiveWritableObject(obj); t.writeToByteStream(byteStream); return; } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe2.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe2.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe2.java index b328508..3e06892 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe2.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe2.java @@ -27,11 +27,11 @@ import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.ByteStream.RandomAccessOutput; import org.apache.hadoop.hive.serde2.SerDeException; import org.apache.hadoop.hive.serde2.SerDeSpec; -import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; 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.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; @@ -337,7 +337,7 @@ public class LazyBinarySerDe2 extends LazyBinarySerDe { @Override void serialize(RandomAccessOutput byteStream, Object obj, ObjectInspector objInspector, boolean skipLengthPrefix, BooleanRef warnedOnceNullMapKey) { - DateWritable d = ((DateObjectInspector) objInspector).getPrimitiveWritableObject(obj); + DateWritableV2 d = ((DateObjectInspector) objInspector).getPrimitiveWritableObject(obj); LazyBinarySerDe.writeDateToByteStream(byteStream, d); } } @@ -347,7 +347,7 @@ public class LazyBinarySerDe2 extends LazyBinarySerDe { void serialize(RandomAccessOutput byteStream, Object obj, ObjectInspector objInspector, boolean skipLengthPrefix, BooleanRef warnedOnceNullMapKey) { TimestampObjectInspector toi = (TimestampObjectInspector) objInspector; - TimestampWritable t = toi.getPrimitiveWritableObject(obj); + TimestampWritableV2 t = toi.getPrimitiveWritableObject(obj); t.writeToByteStream(byteStream); } } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestamp.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestamp.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestamp.java index fa5461a..724a167 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestamp.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestamp.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.hive.serde2.lazybinary; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef; import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampObjectInspector; @@ -29,17 +29,17 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestamp * */ public class LazyBinaryTimestamp extends - LazyBinaryPrimitive<WritableTimestampObjectInspector, TimestampWritable> { + LazyBinaryPrimitive<WritableTimestampObjectInspector, TimestampWritableV2> { static final Logger LOG = LoggerFactory.getLogger(LazyBinaryTimestamp.class); LazyBinaryTimestamp(WritableTimestampObjectInspector oi) { super(oi); - data = new TimestampWritable(); + data = new TimestampWritableV2(); } LazyBinaryTimestamp(LazyBinaryTimestamp copy) { super(copy); - data = new TimestampWritable(copy.data); + data = new TimestampWritableV2(copy.data); } /** http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java index ee1e2e6..eb028e3 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java @@ -23,7 +23,7 @@ import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.hive.serde2.ByteStream.RandomAccessOutput; import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.lazybinary.objectinspector.LazyBinaryObjectInspectorFactory; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; @@ -203,7 +203,7 @@ public final class LazyBinaryUtils { break; case TIMESTAMP: recordInfo.elementOffset = 0; - recordInfo.elementSize = TimestampWritable.getTotalLength(bytes, offset); + recordInfo.elementSize = TimestampWritableV2.getTotalLength(bytes, offset); break; case TIMESTAMPLOCALTZ: recordInfo.elementOffset = 0; http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java index 340f322..000dfed 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java @@ -26,7 +26,7 @@ import java.util.Deque; import java.util.List; import org.apache.hadoop.hive.serde2.fast.DeserializeRead; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils; import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VInt; import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VLong; @@ -303,7 +303,7 @@ public final class LazyBinaryDeserializeRead extends DeserializeRead { break; case TIMESTAMP: { - int length = TimestampWritable.getTotalLength(bytes, offset); + int length = TimestampWritableV2.getTotalLength(bytes, offset); int saveStart = offset; offset += length; // Last item -- ok to be at end. http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java index cd4e619..ec56b82 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java @@ -19,14 +19,14 @@ package org.apache.hadoop.hive.serde2.lazybinary.fast; import java.io.IOException; -import java.sql.Date; -import java.sql.Timestamp; import java.util.ArrayDeque; import java.util.Deque; import java.util.List; import java.util.Map; +import org.apache.hadoop.hive.common.type.Date; import org.apache.hadoop.hive.serde2.ByteStream; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,12 +35,12 @@ import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hadoop.hive.common.type.Timestamp; import org.apache.hadoop.hive.serde2.ByteStream.Output; -import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; 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.lazybinary.LazyBinarySerDe; import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils; import org.apache.hadoop.hive.serde2.fast.SerializeWrite; @@ -64,7 +64,7 @@ public class LazyBinarySerializeWrite implements SerializeWrite { private boolean skipLengthPrefix = false; // For thread safety, we allocate private writable objects for our use only. - private TimestampWritable timestampWritable; + private TimestampWritableV2 timestampWritable; private HiveIntervalYearMonthWritable hiveIntervalYearMonthWritable; private HiveIntervalDayTimeWritable hiveIntervalDayTimeWritable; private HiveIntervalDayTime hiveIntervalDayTime; @@ -308,7 +308,7 @@ public class LazyBinarySerializeWrite implements SerializeWrite { @Override public void writeDate(Date date) throws IOException { beginElement(); - writeVInt(DateWritable.dateToDays(date)); + writeVInt(DateWritableV2.dateToDays(date)); finishElement(); } @@ -327,7 +327,7 @@ public class LazyBinarySerializeWrite implements SerializeWrite { public void writeTimestamp(Timestamp v) throws IOException { beginElement(); if (timestampWritable == null) { - timestampWritable = new TimestampWritable(); + timestampWritable = new TimestampWritableV2(); } timestampWritable.set(v); timestampWritable.writeToByteStream(output); http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java index a442cb1..9393fb8 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java @@ -29,9 +29,9 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import org.apache.hadoop.hive.serde2.ByteStream; -import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableTimestampLocalTZObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampLocalTZObjectInspector; import org.apache.hive.common.util.Murmur3; @@ -39,13 +39,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.SerDeException; -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.lazy.LazyDouble; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions; @@ -720,7 +718,7 @@ public final class ObjectInspectorUtils { case DATE: return ((DateObjectInspector) poi).getPrimitiveWritableObject(o).hashCode(); case TIMESTAMP: - TimestampWritable t = ((TimestampObjectInspector) poi) + TimestampWritableV2 t = ((TimestampObjectInspector) poi) .getPrimitiveWritableObject(o); return t.hashCode(); case TIMESTAMPLOCALTZ: @@ -857,7 +855,7 @@ public final class ObjectInspectorUtils { byteBuffer.putInt(((DateObjectInspector) poi).getPrimitiveWritableObject(o).getDays()); return Murmur3.hash32(byteBuffer.array(), 4); case TIMESTAMP: { - TimestampWritable t = ((TimestampObjectInspector) poi) + TimestampWritableV2 t = ((TimestampObjectInspector) poi) .getPrimitiveWritableObject(o); return Murmur3.hash32(t.getBytes()); } @@ -1112,16 +1110,16 @@ public final class ObjectInspectorUtils { } case DATE: { - DateWritable d1 = ((DateObjectInspector) poi1) + DateWritableV2 d1 = ((DateObjectInspector) poi1) .getPrimitiveWritableObject(o1); - DateWritable d2 = ((DateObjectInspector) poi2) + DateWritableV2 d2 = ((DateObjectInspector) poi2) .getPrimitiveWritableObject(o2); return d1.compareTo(d2); } case TIMESTAMP: { - TimestampWritable t1 = ((TimestampObjectInspector) poi1) + TimestampWritableV2 t1 = ((TimestampObjectInspector) poi1) .getPrimitiveWritableObject(o1); - TimestampWritable t2 = ((TimestampObjectInspector) poi2) + TimestampWritableV2 t2 = ((TimestampObjectInspector) poi2) .getPrimitiveWritableObject(o2); return t1.compareTo(t2); } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/DateObjectInspector.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/DateObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/DateObjectInspector.java index 93a18f7..f58364c 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/DateObjectInspector.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/DateObjectInspector.java @@ -17,11 +17,10 @@ */ package org.apache.hadoop.hive.serde2.objectinspector.primitive; -import java.sql.Date; - import org.apache.hadoop.hive.common.classification.InterfaceAudience; import org.apache.hadoop.hive.common.classification.InterfaceStability; -import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.common.type.Date; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; /** @@ -31,7 +30,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; @InterfaceStability.Stable public interface DateObjectInspector extends PrimitiveObjectInspector { - DateWritable getPrimitiveWritableObject(Object o); + DateWritableV2 getPrimitiveWritableObject(Object o); Date getPrimitiveJavaObject(Object o); } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java index bd86c22..7dc3d07 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java @@ -17,9 +17,8 @@ */ 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.common.type.Date; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; public class JavaConstantDateObjectInspector extends JavaDateObjectInspector @@ -36,6 +35,6 @@ public class JavaConstantDateObjectInspector extends JavaDateObjectInspector if (value==null) { return null; } - return new DateWritable(value); + return new DateWritableV2(value); } } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java index 2453bc6..4da7299 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java @@ -17,9 +17,8 @@ */ 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.common.type.Timestamp; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; public class JavaConstantTimestampObjectInspector extends @@ -36,6 +35,6 @@ public class JavaConstantTimestampObjectInspector extends if (value==null) { return null; } - return new TimestampWritable(value); + return new TimestampWritableV2(value); } } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaDateObjectInspector.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaDateObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaDateObjectInspector.java index d93d719..4cf0a60 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaDateObjectInspector.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaDateObjectInspector.java @@ -17,9 +17,8 @@ */ 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.common.type.Date; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; /** @@ -33,8 +32,8 @@ public class JavaDateObjectInspector super(TypeInfoFactory.dateTypeInfo); } - public DateWritable getPrimitiveWritableObject(Object o) { - return o == null ? null : new DateWritable((Date) o); + public DateWritableV2 getPrimitiveWritableObject(Object o) { + return o == null ? null : new DateWritableV2((Date) o); } @Override @@ -50,20 +49,34 @@ public class JavaDateObjectInspector if (value == null) { return null; } - ((Date) o).setTime(value.getTime()); + ((Date) o).setTimeInDays(value.toEpochDay()); return o; } - public Object set(Object o, DateWritable d) { + @Deprecated + public Object set(Object o, java.sql.Date value) { + if (value == null) { + return null; + } + ((Date) o).setTimeInMillis(value.getTime()); + return o; + } + + public Object set(Object o, DateWritableV2 d) { if (d == null) { return null; } - ((Date) o).setTime(d.get().getTime()); + ((Date) o).setTimeInDays(d.get().toEpochDay()); return o; } + @Deprecated + public Object create(java.sql.Date value) { + return Date.ofEpochMilli(value.getTime()); + } + public Object create(Date value) { - return new Date(value.getTime()); + return Date.ofEpochDay(value.toEpochDay()); } } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampObjectInspector.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampObjectInspector.java index 1e805ba..47719c8 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampObjectInspector.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampObjectInspector.java @@ -17,9 +17,8 @@ */ 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.common.type.Timestamp; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; public class JavaTimestampObjectInspector @@ -30,8 +29,8 @@ public class JavaTimestampObjectInspector super(TypeInfoFactory.timestampTypeInfo); } - public TimestampWritable getPrimitiveWritableObject(Object o) { - return o == null ? null : new TimestampWritable((Timestamp) o); + public TimestampWritableV2 getPrimitiveWritableObject(Object o) { + return o == null ? null : new TimestampWritableV2((Timestamp) o); } @Override @@ -45,43 +44,54 @@ public class JavaTimestampObjectInspector return null; } Timestamp source = (Timestamp) o; - Timestamp copy = new Timestamp(source.getTime()); - copy.setNanos(source.getNanos()); - return copy; + return new Timestamp(source); } public Timestamp get(Object o) { return (Timestamp) o; } + @Deprecated + public Object set(Object o, java.sql.Timestamp value) { + if (value == null) { + return null; + } + ((Timestamp) o).setTimeInMillis(value.getTime(), value.getNanos()); + return o; + } + public Object set(Object o, Timestamp value) { if (value == null) { return null; } - ((Timestamp) o).setTime(value.getTime()); + ((Timestamp) o).set(value); return o; } public Object set(Object o, byte[] bytes, int offset) { - TimestampWritable.setTimestamp((Timestamp) o, bytes, offset); + TimestampWritableV2.setTimestamp((Timestamp) o, bytes, offset); return o; } - public Object set(Object o, TimestampWritable tw) { + public Object set(Object o, TimestampWritableV2 tw) { if (tw == null) { return null; } Timestamp t = (Timestamp) o; - t.setTime(tw.getTimestamp().getTime()); - t.setNanos(tw.getTimestamp().getNanos()); + t.set(tw.getTimestamp()); return t; } + @Deprecated + public Object create(java.sql.Timestamp value) { + return Timestamp.ofEpochMilli(value.getTime(), value.getNanos()); + } + public Object create(Timestamp value) { - return new Timestamp(value.getTime()); + return new Timestamp(value); } public Object create(byte[] bytes, int offset) { - return TimestampWritable.createTimestamp(bytes, offset); + return TimestampWritableV2.createTimestamp(bytes, offset); } } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java index ba20a2c..1e12cca 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java @@ -18,15 +18,15 @@ package org.apache.hadoop.hive.serde2.objectinspector.primitive; -import java.sql.Date; -import java.sql.Timestamp; import java.time.ZoneId; +import org.apache.hadoop.hive.common.type.Date; 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.common.type.Timestamp; import org.apache.hadoop.hive.common.type.TimestampTZ; import org.apache.hadoop.hive.serde2.ByteStream; import org.apache.hadoop.hive.serde2.lazy.LazyInteger; @@ -255,7 +255,7 @@ public class PrimitiveObjectInspectorConverter { SettableDateObjectInspector outputOI) { this.inputOI = inputOI; this.outputOI = outputOI; - r = outputOI.create(new Date(0)); + r = outputOI.create(new Date()); } public Object convert(Object input) { @@ -277,7 +277,7 @@ public class PrimitiveObjectInspectorConverter { SettableTimestampObjectInspector outputOI) { this.inputOI = inputOI; this.outputOI = outputOI; - r = outputOI.create(new Timestamp(0)); + r = outputOI.create(new Timestamp()); } public void setIntToTimestampInSeconds(boolean intToTimestampInSeconds) { http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java index 10af3dc..51a0aed 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java @@ -24,7 +24,7 @@ import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.io.ByteWritable; -import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; import org.apache.hadoop.hive.serde2.io.DoubleWritable; import org.apache.hadoop.hive.serde2.io.HiveCharWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable; import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable; import org.apache.hadoop.hive.serde2.io.ShortWritable; import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory; @@ -346,9 +346,9 @@ public final class PrimitiveObjectInspectorFactory { return new WritableConstantHiveVarcharObjectInspector((VarcharTypeInfo)typeInfo, (HiveVarcharWritable)value); case DATE: - return new WritableConstantDateObjectInspector((DateWritable)value); + return new WritableConstantDateObjectInspector((DateWritableV2)value); case TIMESTAMP: - return new WritableConstantTimestampObjectInspector((TimestampWritable)value); + return new WritableConstantTimestampObjectInspector((TimestampWritableV2)value); case TIMESTAMPLOCALTZ: return new WritableConstantTimestampLocalTZObjectInspector((TimestampLocalTZTypeInfo)typeInfo, (TimestampLocalTZWritable) value); case INTERVAL_YEAR_MONTH: http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java index 8cf0744..6362f2e 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java @@ -23,8 +23,6 @@ import java.io.DataOutput; import java.io.IOException; import java.nio.charset.CharacterCodingException; import java.nio.charset.StandardCharsets; -import java.sql.Date; -import java.sql.Timestamp; import java.time.DateTimeException; import java.time.ZoneId; import java.util.HashMap; @@ -32,18 +30,19 @@ import java.util.Map; import org.apache.hadoop.hive.common.classification.InterfaceAudience; import org.apache.hadoop.hive.common.classification.InterfaceStability; -import org.apache.hadoop.hive.common.type.TimestampTZ; -import org.apache.hadoop.hive.common.type.TimestampTZUtil; -import org.apache.hadoop.hive.ql.util.TimestampUtils; -import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable; +import org.apache.hadoop.hive.common.type.Date; 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.HiveIntervalYearMonth; import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hadoop.hive.common.type.Timestamp; +import org.apache.hadoop.hive.common.type.TimestampTZ; +import org.apache.hadoop.hive.common.type.TimestampTZUtil; +import org.apache.hadoop.hive.common.type.TimestampUtils; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.io.ByteWritable; -import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; import org.apache.hadoop.hive.serde2.io.DoubleWritable; import org.apache.hadoop.hive.serde2.io.HiveCharWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; @@ -51,7 +50,8 @@ 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.ShortWritable; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.lazy.LazyInteger; import org.apache.hadoop.hive.serde2.lazy.LazyLong; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; @@ -229,10 +229,10 @@ public final class PrimitiveObjectInspectorUtils { Short.class, ShortWritable.class); public static final PrimitiveTypeEntry dateTypeEntry = new PrimitiveTypeEntry( PrimitiveCategory.DATE, serdeConstants.DATE_TYPE_NAME, null, - Date.class, DateWritable.class); + Date.class, DateWritableV2.class); public static final PrimitiveTypeEntry timestampTypeEntry = new PrimitiveTypeEntry( PrimitiveCategory.TIMESTAMP, serdeConstants.TIMESTAMP_TYPE_NAME, null, - Timestamp.class, TimestampWritable.class); + Timestamp.class, TimestampWritableV2.class); public static final PrimitiveTypeEntry timestampTZTypeEntry = new PrimitiveTypeEntry( PrimitiveCategory.TIMESTAMPLOCALTZ, serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME, null, TimestampTZ.class, TimestampLocalTZWritable.class); @@ -1126,7 +1126,7 @@ public final class PrimitiveObjectInspectorUtils { } catch (IllegalArgumentException e) { Timestamp ts = getTimestampFromString(s); if (ts != null) { - result = new Date(ts.getTime()); + result = Date.ofEpochMilli(ts.toEpochMilli()); } else { result = null; } @@ -1140,7 +1140,7 @@ public final class PrimitiveObjectInspectorUtils { } catch (IllegalArgumentException e) { Timestamp ts = getTimestampFromString(val); if (ts != null) { - result = new Date(ts.getTime()); + result = Date.ofEpochMilli(ts.toEpochMilli()); } else { result = null; } @@ -1151,7 +1151,7 @@ public final class PrimitiveObjectInspectorUtils { result = ((DateObjectInspector) oi).getPrimitiveWritableObject(o).get(); break; case TIMESTAMP: - result = DateWritable.timeToDate( + result = DateWritableV2.timeToDate( ((TimestampObjectInspector) oi).getPrimitiveWritableObject(o).getSeconds()); break; case TIMESTAMPLOCALTZ: @@ -1187,23 +1187,23 @@ public final class PrimitiveObjectInspectorUtils { break; case BOOLEAN: longValue = ((BooleanObjectInspector) inputOI).get(o) ? 1 : 0; - result = TimestampWritable.longToTimestamp(longValue, intToTimestampInSeconds); + result = TimestampWritableV2.longToTimestamp(longValue, intToTimestampInSeconds); break; case BYTE: longValue = ((ByteObjectInspector) inputOI).get(o); - result = TimestampWritable.longToTimestamp(longValue, intToTimestampInSeconds); + result = TimestampWritableV2.longToTimestamp(longValue, intToTimestampInSeconds); break; case SHORT: longValue = ((ShortObjectInspector) inputOI).get(o); - result = TimestampWritable.longToTimestamp(longValue, intToTimestampInSeconds); + result = TimestampWritableV2.longToTimestamp(longValue, intToTimestampInSeconds); break; case INT: longValue = ((IntObjectInspector) inputOI).get(o); - result = TimestampWritable.longToTimestamp(longValue, intToTimestampInSeconds); + result = TimestampWritableV2.longToTimestamp(longValue, intToTimestampInSeconds); break; case LONG: longValue = ((LongObjectInspector) inputOI).get(o); - result = TimestampWritable.longToTimestamp(longValue, intToTimestampInSeconds); + result = TimestampWritableV2.longToTimestamp(longValue, intToTimestampInSeconds); break; case FLOAT: result = TimestampUtils.doubleToTimestamp(((FloatObjectInspector) inputOI).get(o)); @@ -1212,8 +1212,8 @@ public final class PrimitiveObjectInspectorUtils { result = TimestampUtils.doubleToTimestamp(((DoubleObjectInspector) inputOI).get(o)); break; case DECIMAL: - result = TimestampUtils.decimalToTimestamp(((HiveDecimalObjectInspector) inputOI) - .getPrimitiveJavaObject(o)); + result = TimestampUtils.decimalToTimestamp( + ((HiveDecimalObjectInspector) inputOI).getPrimitiveJavaObject(o)); break; case STRING: StringObjectInspector soi = (StringObjectInspector) inputOI; @@ -1225,8 +1225,8 @@ public final class PrimitiveObjectInspectorUtils { result = getTimestampFromString(getString(o, inputOI)); break; case DATE: - result = new Timestamp( - ((DateObjectInspector) inputOI).getPrimitiveWritableObject(o).get().getTime()); + result = Timestamp.ofEpochMilli( + ((DateObjectInspector) inputOI).getPrimitiveWritableObject(o).get().toEpochMilli()); break; case TIMESTAMP: result = ((TimestampObjectInspector) inputOI).getPrimitiveWritableObject(o).getTimestamp(); @@ -1247,23 +1247,25 @@ public final class PrimitiveObjectInspectorUtils { return result; } - static Timestamp getTimestampFromString(String s) { + public static Timestamp getTimestampFromString(String s) { Timestamp result; s = s.trim(); s = trimNanoTimestamp(s); - int firstSpace = s.indexOf(' '); - if (firstSpace < 0) { - s = s.concat(" 00:00:00"); - } try { result = Timestamp.valueOf(s); } catch (IllegalArgumentException e) { // Let's try to parse it as timestamp with time zone and transform try { - result = Timestamp.from(TimestampTZUtil.parse(s).getZonedDateTime().toInstant()); + result = Timestamp.valueOf(TimestampTZUtil.parse(s).getZonedDateTime() + .toLocalDateTime().toString()); } catch (DateTimeException e2) { - result = null; + // Last try: we try to parse it as date and transform + try { + result = Timestamp.ofEpochMilli(Date.valueOf(s).toEpochMilli()); + } catch (IllegalArgumentException e3) { + result = null; + } } } return result; http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableDateObjectInspector.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableDateObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableDateObjectInspector.java index 831411d..725d5cd 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableDateObjectInspector.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableDateObjectInspector.java @@ -17,17 +17,22 @@ */ 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.common.type.Date; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; /** * A SettableDecimalObjectInspector can set a Date value to an object. */ public interface SettableDateObjectInspector extends DateObjectInspector { + @Deprecated + Object set(Object o, java.sql.Date d); + Object set(Object o, Date d); - Object set(Object o, DateWritable d); + Object set(Object o, DateWritableV2 d); + + @Deprecated + Object create(java.sql.Date d); Object create(Date d); } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampObjectInspector.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampObjectInspector.java index c676a62..65cae16 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampObjectInspector.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampObjectInspector.java @@ -17,20 +17,25 @@ */ 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.common.type.Timestamp; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; public interface SettableTimestampObjectInspector extends TimestampObjectInspector { Object set(Object o, byte[] bytes, int offset); + @Deprecated + Object set(Object o, java.sql.Timestamp t); + Object set(Object o, Timestamp t); - Object set(Object o, TimestampWritable t); + Object set(Object o, TimestampWritableV2 t); Object create(byte[] bytes, int offset); + @Deprecated + Object create (java.sql.Timestamp t); + Object create (Timestamp t); } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampObjectInspector.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampObjectInspector.java index f277232..6eb2aac 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampObjectInspector.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampObjectInspector.java @@ -17,18 +17,17 @@ */ package org.apache.hadoop.hive.serde2.objectinspector.primitive; -import java.sql.Timestamp; - import org.apache.hadoop.hive.common.classification.InterfaceAudience; import org.apache.hadoop.hive.common.classification.InterfaceStability; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.common.type.Timestamp; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; @InterfaceAudience.Public @InterfaceStability.Stable public interface TimestampObjectInspector extends PrimitiveObjectInspector { - TimestampWritable getPrimitiveWritableObject(Object o); + TimestampWritableV2 getPrimitiveWritableObject(Object o); Timestamp getPrimitiveJavaObject(Object o); } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantDateObjectInspector.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantDateObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantDateObjectInspector.java index 290fcd3..7e6cc8d 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantDateObjectInspector.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantDateObjectInspector.java @@ -17,10 +17,9 @@ */ package org.apache.hadoop.hive.serde2.objectinspector.primitive; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; -import org.apache.hadoop.hive.serde2.io.DateWritable; - /** * A WritableConstantDateObjectInspector is a WritableDateObjectInspector * that implements ConstantObjectInspector. @@ -29,18 +28,18 @@ public class WritableConstantDateObjectInspector extends WritableDateObjectInspector implements ConstantObjectInspector { - private DateWritable value; + private DateWritableV2 value; protected WritableConstantDateObjectInspector() { super(); } - WritableConstantDateObjectInspector(DateWritable value) { + WritableConstantDateObjectInspector(DateWritableV2 value) { super(); this.value = value; } @Override - public DateWritable getWritableConstantValue() { + public DateWritableV2 getWritableConstantValue() { return value; } } http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampObjectInspector.java ---------------------------------------------------------------------- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampObjectInspector.java index dc8fedf..9428421 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampObjectInspector.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampObjectInspector.java @@ -17,10 +17,9 @@ */ package org.apache.hadoop.hive.serde2.objectinspector.primitive; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; - /** * A WritableConstantTimestampObjectInspector is a WritableTimestampObjectInspector * that implements ConstantObjectInspector. @@ -29,18 +28,18 @@ public class WritableConstantTimestampObjectInspector extends WritableTimestampObjectInspector implements ConstantObjectInspector { - private TimestampWritable value; + private TimestampWritableV2 value; protected WritableConstantTimestampObjectInspector() { super(); } - WritableConstantTimestampObjectInspector(TimestampWritable value) { + WritableConstantTimestampObjectInspector(TimestampWritableV2 value) { super(); this.value = value; } @Override - public TimestampWritable getWritableConstantValue() { + public TimestampWritableV2 getWritableConstantValue() { return value; } }