diff --git a/build-common.xml b/build-common.xml index 9f21a69..192b127 100644 --- a/build-common.xml +++ b/build-common.xml @@ -446,6 +446,7 @@ + diff --git a/data/files/test_v6_compressed.rc b/data/files/test_v6_compressed.rc new file mode 100755 index 0000000000000000000000000000000000000000..8cd914cc4bc18ed0635a357404be2dacc5f2e422 GIT binary patch literal 252 zcmWG`4P?{JFG|--EJ#ewNY%?oOv%qL(96gyOVumP(aX%&3vzbL%t=-8POWq*O-oBH z(kDiBSYl3TDnvaagBGR^pt;HUxdlb3#l?CqscDI&IVH~dDXGZ}3=E7iV9S9jijvcS zHUkyrl;-B?<(1|pr4}(7WqPEnfAeDh?Npf;mw1@|v40x>@ji&V@>ewdt`n39z8 oh%xZL1S3lXqjkj`Zyzr&A1@{b4t6H*iaEw6Oia!UERu}c0Ep33{Qv*} literal 0 HcmV?d00001 diff --git a/data/files/test_v6_uncompressed.rc b/data/files/test_v6_uncompressed.rc new file mode 100755 index 0000000000000000000000000000000000000000..1232d28503da91e6aa4882b33fc04659af0e3e11 GIT binary patch literal 190 zcmWG`4P?{JFG|--EJ#ewNY%?oOv%qL(96gyOVumP(aX%&3vzbL%t=-8POWq*O-oBH z(kDiBSYl3TDnva42r$ZkO#~WOl$-`MPA@q>r!+TDFRwH=DYb~v=(gw9#C&xRd)Kw* lJofK|&j7VZ12I1cv#_zTFfuVRfvJCtiAhO7WM~8gh5(^-IaL4v literal 0 HcmV?d00001 diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java b/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java index eb5305b..d7c19dc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java @@ -73,7 +73,7 @@ import org.apache.hadoop.util.ReflectionUtils; 4 * 1024 * 1024) . *

* RCFile provides {@link Writer}, {@link Reader} and classes for - * writing, reading respectively. + * writing, reading respectively. These classes are not thread-safe. *

* *

@@ -86,9 +86,8 @@ import org.apache.hadoop.util.ReflectionUtils; * *

* RCFile compresses values in a more fine-grained manner then record level - * compression. However, It currently does not support compress the key part - * yet. The actual compression algorithm used to compress key and/or values can - * be specified by using the appropriate {@link CompressionCodec}. + * compression. The actual compression algorithm used to compress key and/or + * values can be specified by using the appropriate {@link CompressionCodec}. *

* *

@@ -102,46 +101,68 @@ import org.apache.hadoop.util.ReflectionUtils; *

    *
  • version - 3 bytes of magic header SEQ, followed by 1 byte of * actual version number (e.g. SEQ4 or SEQ6)
  • - *
  • keyClassName -KeyBuffer's class name
  • + *
  • keyClassName - KeyBuffer's class name
  • *
  • valueClassName - ValueBuffer's class name
  • - *
  • compression - A boolean which specifies if compression is turned on for - * keys/values in this file.
  • + *
  • compression - always false. this field is kept for compatible with + * SequeceFile's format
  • *
  • blockCompression - always false. this field is kept for compatible with * SequeceFile's format
  • - *
  • compression codec - CompressionCodec class which is used for - * compression of keys and/or values (if compression is enabled).
  • *
  • metadata - {@link Metadata} for this file.
  • + *
      + *
    • compression codec - CompressionCodec class which is used for + * compression of keys and/or values (if compression is enabled).
    • + *
    • minor version - The minor version of the file format
    • + *
    • number of columns - the number of columns in the data
    • + *
    *
  • sync - A sync marker to denote end of the header.
  • *
* *
RCFile Format
*
    - *
  • Header
  • - *
  • Record - *
  • Key part - *
      - *
    • Record length in bytes
    • - *
    • Key length in bytes
    • - *
    • Number_of_rows_in_this_record(vint)
    • - *
    • Column_1_ondisk_length(vint)
    • - *
    • Column_1_row_1_value_plain_length
    • - *
    • Column_1_row_2_value_plain_length
    • - *
    • ...
    • - *
    • Column_2_ondisk_length(vint)
    • - *
    • Column_2_row_1_value_plain_length
    • - *
    • Column_2_row_2_value_plain_length
    • - *
    • ...
    • - *
    - *
  • - * - *
  • Value part - *
      - *
    • Compressed or plain data of [column_1_row_1_value, - * column_1_row_2_value,....]
    • - *
    • Compressed or plain data of [column_2_row_1_value, - * column_2_row_2_value,....]
    • - *
    - *
  • + *
  • Header + *
  • Records, one or more + *
      + *
    • Record Header + *
        + *
      • Record Body length in bytes + *
      • Key length in bytes + *
      + *
    • Record Body + *
        + *
      • Key + *
          + *
        • Key contents' length in bytes + *
        • Key contents, plain or compressed + *
            + *
          • Number_of_rows_in_this_record(vint) + *
          • Column_1_ondisk_length(vint) + *
          • Column_1_plain_length(vint) + *
          • RL-encoded Column_1_row_1_value_plain_length + *
          • RL-encoded Column_1_row_2_value_plain_length + *
          • ... + *
          • Column_2_ondisk_length(vint) + *
          • Column_2_plain_length(vint) + *
          • RL-encoded Column_2_row_1_value_plain_length + *
          • RL-encoded Column_2_row_2_value_plain_length + *
          • ... + *
          + *
        + *
      • Value + *
          + *
        • Compressed or plain data of [column_1_row_1_value, + * column_1_row_2_value,....] + *
        • Compressed or plain data of [column_2_row_1_value, + * column_2_row_2_value,....] + *
        + *
      + *
    + *
  • Sync block + *
      + *
    • sync escape + *
    • sync + *
    + *
  • Records, one or more + *
  • ... *
* */ @@ -153,17 +174,23 @@ public class RCFile { public static final String COLUMN_NUMBER_METADATA_STR = "hive.io.rcfile.column.number"; + public static final String COMPRESSION_CODEC_METADATA_STR = "hive.io.rcfile.compression.codec"; + + public static final String RCFILE_VERSION_METADATA_STR = "hive.io.rcfile.minor.version"; + public static final String COLUMN_NUMBER_CONF_STR = "hive.io.rcfile.column.number.conf"; /* * these header and Sync are kept from SequenceFile, for compatible of * SequenceFile's format. */ - private static final byte VERSION_WITH_METADATA = (byte) 6; + public static final byte VERSION_WITH_METADATA = (byte) 6; private static final byte[] VERSION = new byte[] { (byte) 'S', (byte) 'E', (byte) 'Q', VERSION_WITH_METADATA }; - + + private static final byte RCFILE_MINOR_VERSION = (byte) 1; + private static final int SYNC_ESCAPE = -1; // "length" of sync entries private static final int SYNC_HASH_SIZE = 16; // number of bytes in hash private static final int SYNC_SIZE = 4 + SYNC_HASH_SIZE; // escape + hash @@ -171,25 +198,11 @@ public class RCFile { /** The number of bytes between sync points. */ public static final int SYNC_INTERVAL = 100 * SYNC_SIZE; + public static final int INT_SIZE = Integer.SIZE/8; + /** - * KeyBuffer is the key of each record in RCFile. Its on-disk layout is as - * below: - * - *
    - *
  • record length in bytes,it is the sum of bytes used to store the key - * part and the value part.
  • - *
  • Key length in bytes, it is how many bytes used by the key part.
  • - *
  • number_of_rows_in_this_record(vint),
  • - *
  • column_1_ondisk_length(vint),
  • - *
  • column_1_row_1_value_plain_length,
  • - *
  • column_1_row_2_value_plain_length,
  • - *
  • ....
  • - *
  • column_2_ondisk_length(vint),
  • - *
  • column_2_row_1_value_plain_length,
  • - *
  • column_2_row_2_value_plain_length,
  • - *
  • .... .
  • - *
  • {the end of the key part}
  • - *
+ * KeyBuffer is the key of each record in RCFile. Its on-disk layout is + * described in the "key" section layout in the {@link RCFile} layout. */ public static class KeyBuffer implements WritableComparable { // each column's value length in a split @@ -207,21 +220,74 @@ public class RCFile { return columnNumber; } - public KeyBuffer(){ - } - - KeyBuffer(int columnNumber) { - this(0, columnNumber); - } - - KeyBuffer(int numberRows, int columnNum) { - columnNumber = columnNum; + private KeyBuffer(int columnNumber){ + this.columnNumber = columnNumber; eachColumnValueLen = new int[columnNumber]; eachColumnUncompressedValueLen = new int[columnNumber]; allCellValLenBuffer = new NonSyncDataOutputBuffer[columnNumber]; - this.numberRows = numberRows; } + CompressionCodec codec; + + /** fields specific to reading the key */ + int majorVersion; + int minorVersion; + int currentRecordLength; + int currentKeyLength; + + int plainKeyContentsLength; + Decompressor keyDecompressor; + CompressionInputStream deflatFilter; + + NonSyncDataInputBuffer keyDataIn; + NonSyncDataInputBuffer keyDecompressBuffer; + NonSyncDataOutputBuffer keyTempBuffer; + NonSyncDataOutputBuffer keyDecompressedData; + + /** The ctor to use for reading the key + * @throws IOException */ + KeyBuffer(int majorversion, int minorversion, + int columnNumber, CompressionCodec codec) throws IOException { + this(columnNumber); + this.majorVersion = majorversion; + this.minorVersion = minorversion; + if (codec != null) { + this.codec = codec; + keyDecompressor = CodecPool.getDecompressor(codec); + keyDataIn = new NonSyncDataInputBuffer(); + keyDecompressBuffer = new NonSyncDataInputBuffer(); + keyTempBuffer = new NonSyncDataOutputBuffer(); + keyDecompressedData = new NonSyncDataOutputBuffer(); + deflatFilter = codec.createInputStream( + keyDecompressBuffer, keyDecompressor); + } + } + + /** fields specific to writing the key */ + Compressor keyCompressor; + NonSyncDataOutputBuffer keyCompressionBuffer; + CompressionOutputStream keyDeflateFilter; + DataOutputStream keyDeflateOut; + + /** The ctor to use for writing the key */ + KeyBuffer(int columnNumber, CompressionCodec codec) throws IOException { + this(columnNumber); + if (codec != null) { + this.codec = codec; + keyCompressor = CodecPool.getCompressor(codec); + keyCompressionBuffer = new NonSyncDataOutputBuffer(); + keyDeflateFilter = codec.createOutputStream(keyCompressionBuffer, + keyCompressor); + keyDeflateOut = new DataOutputStream(new BufferedOutputStream( + keyDeflateFilter)); + } + } + + void setRecordInfo(int currentRecordLength, int currentKeyLength) { + this.currentRecordLength = currentRecordLength; + this.currentKeyLength = currentKeyLength; + } + /** * add in a new column's meta data. * @@ -240,27 +306,81 @@ public class RCFile { @Override public void readFields(DataInput in) throws IOException { - eachColumnValueLen = new int[columnNumber]; - eachColumnUncompressedValueLen = new int[columnNumber]; - allCellValLenBuffer = new NonSyncDataOutputBuffer[columnNumber]; - - numberRows = WritableUtils.readVInt(in); + int plainKeyContentsLength = in.readInt(); + if ((majorVersion < 6) || (majorVersion == 6) && (minorVersion < 1)) { + //version 6.0 and below had incorrect record length plus + //uncompressed key contents length as the next field + int temp = plainKeyContentsLength; + plainKeyContentsLength = currentKeyLength; + currentKeyLength = INT_SIZE + temp; + currentRecordLength = currentRecordLength - plainKeyContentsLength + + currentKeyLength; + } + DataInput dataInput = in; + if (keyDecompressor != null) { + int compressedKeyContentsLength = currentKeyLength - INT_SIZE; + keyTempBuffer.reset(); + keyTempBuffer.write(in, compressedKeyContentsLength); + keyDecompressBuffer.reset(keyTempBuffer.getData(), compressedKeyContentsLength); + DataInputStream compressedIn = new DataInputStream(deflatFilter); + deflatFilter.resetState(); + keyDecompressedData.reset(); + keyDecompressedData.write(compressedIn, plainKeyContentsLength); + keyDataIn.reset(keyDecompressedData.getData(), plainKeyContentsLength); + dataInput = keyDataIn; + } + + numberRows = WritableUtils.readVInt(dataInput); for (int i = 0; i < columnNumber; i++) { - eachColumnValueLen[i] = WritableUtils.readVInt(in); - eachColumnUncompressedValueLen[i] = WritableUtils.readVInt(in); - int bufLen = WritableUtils.readVInt(in); + eachColumnValueLen[i] = WritableUtils.readVInt(dataInput); + eachColumnUncompressedValueLen[i] = WritableUtils.readVInt(dataInput); + int bufLen = WritableUtils.readVInt(dataInput); if (allCellValLenBuffer[i] == null) { allCellValLenBuffer[i] = new NonSyncDataOutputBuffer(); } else { allCellValLenBuffer[i].reset(); } - allCellValLenBuffer[i].write(in, bufLen); + allCellValLenBuffer[i].write(dataInput, bufLen); } } - + + public int prepareToWrite() throws IOException { + int plainkeyContentsLength = getSize(); + if (plainkeyContentsLength < 0) { + throw new IOException("negative length keys not allowed: " + this); + } + int compressedKeyContentsLen = plainkeyContentsLength; + if (codec != null) { + //compress key and write key out + if (keyCompressor == null) { + keyCompressor = CodecPool.getCompressor(codec); + keyCompressionBuffer = new NonSyncDataOutputBuffer(); + keyDeflateFilter = codec.createOutputStream(keyCompressionBuffer, + keyCompressor); + keyDeflateOut = new DataOutputStream(new BufferedOutputStream( + keyDeflateFilter)); + } + doWrite(keyDeflateOut); + keyDeflateOut.flush(); + keyDeflateFilter.finish(); + compressedKeyContentsLen = keyCompressionBuffer.getLength(); + } + return INT_SIZE + compressedKeyContentsLen; + } + + @Override - public void write(DataOutput out) throws IOException { - // out.writeInt(numberRows); + public void write(DataOutput out) throws IOException { + out.writeInt(getSize()); + if (keyCompressor == null) { + doWrite(out); // key contents + } else { + out.write(keyCompressionBuffer.getData(), 0, keyCompressionBuffer.getLength()); + } + + } + + private void doWrite(DataOutput out) throws IOException { WritableUtils.writeVLong(out, numberRows); for (int i = 0; i < eachColumnValueLen.length; i++) { WritableUtils.writeVLong(out, eachColumnValueLen[i]); @@ -271,14 +391,14 @@ public class RCFile { out.write(colRowsLenBuf.getData(), 0, bufLen); } } - + /** - * get number of bytes to store the keyBuffer. + * get number of bytes to store the key contents uncompressed. * - * @return number of bytes used to store this KeyBuffer on disk + * @return number of bytes to store the key contents uncompressed * @throws IOException */ - public int getSize() throws IOException { + private int getSize() throws IOException { int ret = 0; ret += WritableUtils.getVIntSize(numberRows); for (int i = 0; i < eachColumnValueLen.length; i++) { @@ -287,7 +407,6 @@ public class RCFile { ret += WritableUtils.getVIntSize(allCellValLenBuffer[i].getLength()); ret += allCellValLenBuffer[i].getLength(); } - return ret; } @@ -296,6 +415,25 @@ public class RCFile { throw new RuntimeException("compareTo not supported in class " + this.getClass().getName()); } + + public void close() throws IOException { + if (keyDecompressedData != null) { + IOUtils.closeStream(keyDecompressedData); + } + if (keyDecompressor != null) { + CodecPool.returnDecompressor(keyDecompressor); + } + if (keyDeflateFilter != null) { + keyDeflateFilter.close(); + } + if (keyDeflateOut != null) { + IOUtils.closeStream(keyDeflateOut); + } + if (keyCompressor != null) { + CodecPool.returnCompressor(keyCompressor); + } + } + } /** @@ -488,6 +626,14 @@ public class RCFile { throw new RuntimeException("compareTo not supported in class " + this.getClass().getName()); } + + public int getSize() { + int valueSize = 0; + for (NonSyncDataOutputBuffer currentBuf : loadedColumnsValueBuffer) { + valueSize += currentBuf.getLength(); + } + return valueSize; + } } /** @@ -535,10 +681,6 @@ public class RCFile { DataOutputStream[] deflateOut = null; private final ColumnBuffer[] columnBuffers; - NonSyncDataOutputBuffer keyCompressionBuffer; - CompressionOutputStream keyDeflateFilter; - DataOutputStream keyDeflateOut; - Compressor keyCompressor; private int columnNumber = 0; @@ -702,7 +844,7 @@ public class RCFile { initializeFileHeader(); writeFileHeader(); finalizeFileHeader(); - key = new KeyBuffer(columnNumber); + key = new KeyBuffer(columnNumber, codec); value = new ValueBuffer(key); } @@ -726,12 +868,14 @@ public class RCFile { Text.writeString(out, KeyBuffer.class.getName()); Text.writeString(out, ValueBuffer.class.getName()); - out.writeBoolean(isCompressed()); - out.writeBoolean(false); + out.writeBoolean(false); //record compressed + out.writeBoolean(false); //block compressed if (isCompressed()) { - Text.writeString(out, (codec.getClass()).getName()); + metadata.set(new Text(COMPRESSION_CODEC_METADATA_STR), + new Text(codec.getClass().getName())); } + metadata.set(new Text(RCFILE_VERSION_METADATA_STR), new Text("" + RCFILE_MINOR_VERSION)); metadata.write(out); } @@ -755,12 +899,6 @@ public class RCFile { deflateOut[i] = new DataOutputStream(new BufferedOutputStream( deflateFilter[i])); } - keyCompressor = CodecPool.getCompressor(codec); - keyCompressionBuffer = new NonSyncDataOutputBuffer(); - keyDeflateFilter = codec.createOutputStream(keyCompressionBuffer, - keyCompressor); - keyDeflateOut = new DataOutputStream(new BufferedOutputStream( - keyDeflateFilter)); } } @@ -770,7 +908,7 @@ public class RCFile { } /** create a sync point. */ - public void sync() throws IOException { + private void sync() throws IOException { if (sync != null && lastSyncPos != out.getPos()) { out.writeInt(SYNC_ESCAPE); // mark the start of the sync out.write(sync); // write sync @@ -865,30 +1003,8 @@ public class RCFile { } columnValuePlainLength[columnIndex] = 0; } - - int keyLength = key.getSize(); - if (keyLength < 0) { - throw new IOException("negative length keys not allowed: " + key); - } - - // Write the record out - checkAndWriteSync(); // sync - out.writeInt(keyLength + valueLength); // total record length - out.writeInt(keyLength); // key portion length - if (!isCompressed()) { - out.writeInt(keyLength); - key.write(out); // key - } else { - keyCompressionBuffer.reset(); - keyDeflateFilter.resetState(); - key.write(keyDeflateOut); - keyDeflateOut.flush(); - keyDeflateFilter.finish(); - int compressedKeyLen = keyCompressionBuffer.getLength(); - out.writeInt(compressedKeyLen); - out.write(keyCompressionBuffer.getData(), 0, compressedKeyLen); - } - value.write(out); // value + + flushBlock(key, value, valueLength); // clear the columnBuffers clearColumnBuffers(); @@ -900,27 +1016,16 @@ public class RCFile { /** * flush a block out without doing anything except compressing the key part. */ - public void flushBlock(KeyBuffer keyBuffer, ValueBuffer valueBuffer, - int recordLen, int keyLength, int compressedKeyLen) throws IOException { - checkAndWriteSync(); // sync - out.writeInt(recordLen); // total record length - out.writeInt(keyLength); // key portion length - - if(this.isCompressed()) { - //compress key and write key out - keyCompressionBuffer.reset(); - keyDeflateFilter.resetState(); - keyBuffer.write(keyDeflateOut); - keyDeflateOut.flush(); - keyDeflateFilter.finish(); - compressedKeyLen = keyCompressionBuffer.getLength(); - out.writeInt(compressedKeyLen); - out.write(keyCompressionBuffer.getData(), 0, compressedKeyLen); - } else { - out.writeInt(compressedKeyLen); - keyBuffer.write(out); + public void flushBlock(KeyBuffer keyBuffer, ValueBuffer valueBuffer, int valueLength) + throws IOException { + if (valueLength == 0) { + valueLength = valueBuffer.getSize(); } - + checkAndWriteSync(); // sync + int keyLength = keyBuffer.prepareToWrite(); + out.writeInt(keyLength + valueLength); // total record length + out.writeInt(keyLength); // key length + keyBuffer.write(out); // key valueBuffer.write(out); // value } @@ -930,7 +1035,7 @@ public class RCFile { } } - public synchronized void close() throws IOException { + public void close() throws IOException { if (bufferedRecords > 0) { flushRecords(); } @@ -941,10 +1046,7 @@ public class RCFile { deflateFilter[i].close(); IOUtils.closeStream(deflateOut[i]); } - keyDeflateFilter.close(); - IOUtils.closeStream(keyDeflateOut); - CodecPool.returnCompressor(keyCompressor); - keyCompressor = null; + key.close(); CodecPool.returnCompressor(compressor); compressor = null; } @@ -969,7 +1071,7 @@ public class RCFile { private final FSDataInputStream in; private byte version; - + private byte rcfileMinorVersion; private CompressionCodec codec = null; private Metadata metadata = null; @@ -981,7 +1083,7 @@ public class RCFile { private long headerEnd; private final long end; private int currentKeyLength; - private int currentRecordLength; + private int currentRecordLength = 0; private final Configuration conf; @@ -1005,8 +1107,6 @@ public class RCFile { private final int[] columnPrvLength; private boolean decompress = false; - private Decompressor keyDecompressor; - NonSyncDataOutputBuffer keyDecompressedData = new NonSyncDataOutputBuffer(); int[] prjColIDs = null; // selected column IDs @@ -1141,7 +1241,7 @@ public class RCFile { throw new IOException(file + " not a RCFile", e); } - if (version > 2) { // if version > 2 + if (version > 2) { decompress = in.readBoolean(); // is compressed? } else { decompress = false; @@ -1154,8 +1254,29 @@ public class RCFile { } // setup the compression codec + String codecClassname = null; + if (decompress) { + codecClassname = Text.readString(in); + } + + metadata = new Metadata(); + rcfileMinorVersion = 0; + if (version >= VERSION_WITH_METADATA) { // if version >= 6 + metadata.readFields(in); + Text rcfv = metadata.get(new Text(RCFILE_VERSION_METADATA_STR)); + if (rcfv != null) { + rcfileMinorVersion = (byte)Integer.parseInt(rcfv.toString()); + } + if ((version > 6) || ((version == 6) && (rcfileMinorVersion >= 1))) { // full version > 6.1 + Text t = metadata.get(new Text(COMPRESSION_CODEC_METADATA_STR)); + if (t != null) { + decompress = true; + codecClassname = t.toString(); + } + } + } + if (decompress) { - String codecClassname = Text.readString(in); try { Class codecClass = conf.getClassByName( codecClassname).asSubclass(CompressionCodec.class); @@ -1165,14 +1286,8 @@ public class RCFile { throw new IllegalArgumentException( "Unknown codec: " + codecClassname, cnfe); } - keyDecompressor = CodecPool.getDecompressor(codec); } - - metadata = new Metadata(); - if (version >= VERSION_WITH_METADATA) { // if version >= 6 - metadata.readFields(in); - } - + if (version > 1) { // if version > 1 in.readFully(sync); // read sync bytes headerEnd = in.getPos(); @@ -1180,7 +1295,7 @@ public class RCFile { } /** Return the current byte position in the input file. */ - public synchronized long getPosition() throws IOException { + public long getPosition() throws IOException { return in.getPos(); } @@ -1194,12 +1309,12 @@ public class RCFile { * words, the current seek can only seek to the end of the file. For other * positions, use {@link RCFile.Reader#sync(long)}. */ - public synchronized void seek(long position) throws IOException { + public void seek(long position) throws IOException { in.seek(position); } /** Seek to the next sync mark past a given position. */ - public synchronized void sync(long position) throws IOException { + public void sync(long position) throws IOException { if (position + SYNC_SIZE >= end) { seek(end); return; @@ -1246,8 +1361,9 @@ public class RCFile { } } - private KeyBuffer createKeyBuffer() { - return new KeyBuffer(columnNumber); + private KeyBuffer createKeyBuffer() throws IOException { + return new KeyBuffer(version, rcfileMinorVersion, + columnNumber, codec); } @SuppressWarnings("unused") @@ -1262,7 +1378,7 @@ public class RCFile { * @return the length of the next record or -1 if there is no next record * @throws IOException */ - private synchronized int readRecordLength() throws IOException { + private int readRecordLength() throws IOException { if (in.getPos() >= end) { return -1; } @@ -1295,39 +1411,20 @@ public class RCFile { } } - private int compressedKeyLen = 0; - NonSyncDataInputBuffer keyDataIn = new NonSyncDataInputBuffer(); - NonSyncDataInputBuffer keyDecompressBuffer = new NonSyncDataInputBuffer(); - NonSyncDataOutputBuffer keyTempBuffer = new NonSyncDataOutputBuffer(); - KeyBuffer currentKey = null; boolean keyInit = false; - protected int nextKeyBuffer() throws IOException { + protected boolean nextKeyBuffer() throws IOException { seekToNextKeyBuffer(); currentRecordLength = readRecordLength(); if (currentRecordLength == -1) { keyInit = false; - return -1; + return false; } currentKeyLength = in.readInt(); - compressedKeyLen = in.readInt(); - if (decompress) { - keyTempBuffer.reset(); - keyTempBuffer.write(in, compressedKeyLen); - keyDecompressBuffer.reset(keyTempBuffer.getData(), compressedKeyLen); - CompressionInputStream deflatFilter = codec.createInputStream( - keyDecompressBuffer, keyDecompressor); - DataInputStream compressedIn = new DataInputStream(deflatFilter); - deflatFilter.resetState(); - keyDecompressedData.reset(); - keyDecompressedData.write(compressedIn, currentKeyLength); - keyDataIn.reset(keyDecompressedData.getData(), currentKeyLength); - currentKey.readFields(keyDataIn); - } else { - currentKey.readFields(in); - } - + currentKey.setRecordInfo(currentRecordLength, currentKeyLength); + currentKey.readFields(in); + fixUpforVersion6dot0(); // fixes up incorrect lengths keyInit = true; currentValue.inited = false; @@ -1343,7 +1440,14 @@ public class RCFile { columnPrvLength[i] = -1; } - return currentKeyLength; + return true; + } + + private void fixUpforVersion6dot0() { + if ((version < 6) || (version == 6) && (rcfileMinorVersion < 1)) { + currentRecordLength = currentKey.currentRecordLength; + currentKeyLength = currentKey.currentKeyLength; + } } protected void currentValueBuffer() throws IOException { @@ -1357,8 +1461,8 @@ public class RCFile { } public boolean nextBlock() throws IOException { - int keyLength = nextKeyBuffer(); - if(keyLength > 0) { + boolean ret = nextKeyBuffer(); + if(ret) { currentValueBuffer(); return true; } @@ -1427,9 +1531,9 @@ public class RCFile { * @return whether there still has records or not * @throws IOException */ - public synchronized boolean nextColumnsBatch() throws IOException { + public boolean nextColumnsBatch() throws IOException { passedRowsNum += (recordsNumInValBuffer - readRowsIndexInBuffer); - return nextKeyBuffer() > 0; + return nextKeyBuffer(); } /** @@ -1442,7 +1546,7 @@ public class RCFile { * @return next row number * @throws IOException */ - public synchronized boolean next(LongWritable readRows) throws IOException { + public boolean next(LongWritable readRows) throws IOException { if (hasRecordsInBuffer()) { readRows.set(passedRowsNum); readRowsIndexInBuffer++; @@ -1453,13 +1557,13 @@ public class RCFile { keyInit = false; } - int ret = -1; + boolean ret = false; try { ret = nextKeyBuffer(); } catch (EOFException eof) { eof.printStackTrace(); } - if (ret > 0) { + if (ret) { return next(readRows); } return false; @@ -1475,7 +1579,7 @@ public class RCFile { * * @throws IOException */ - public synchronized void getCurrentRow(BytesRefArrayWritable ret) throws IOException { + public void getCurrentRow(BytesRefArrayWritable ret) throws IOException { if (!keyInit || rowFetched) { return; @@ -1551,14 +1655,12 @@ public class RCFile { return this.decompress; } - /** Close the reader. */ - public void close() { + /** Close the reader. + * @throws IOException */ + public void close() throws IOException { IOUtils.closeStream(in); currentValue.close(); - if (decompress) { - IOUtils.closeStream(keyDecompressedData); - CodecPool.returnDecompressor(keyDecompressor); - } + currentKey.close(); } /** @@ -1579,21 +1681,6 @@ public class RCFile { return this.currentValue; } - //return the current block's length - public int getCurrentBlockLength() { - return this.currentRecordLength; - } - - //return the current block's key length - public int getCurrentKeyLength() { - return this.currentKeyLength; - } - - //return the current block's compressed key length - public int getCurrentCompressedKeyLen() { - return this.compressedKeyLen; - } - //return the CompressionCodec used for this file public CompressionCodec getCompressionCodec() { return this.codec; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/RCFileBlockMergeRecordReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/RCFileBlockMergeRecordReader.java index 20d1f4e..f1d73e1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/RCFileBlockMergeRecordReader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/RCFileBlockMergeRecordReader.java @@ -89,9 +89,6 @@ public class RCFileBlockMergeRecordReader implements } keyWrapper.keyBuffer = this.in.getCurrentKeyBufferObj(); - keyWrapper.recordLength = this.in.getCurrentBlockLength(); - keyWrapper.keyLength = this.in.getCurrentKeyLength(); - keyWrapper.compressedKeyLength = this.in.getCurrentCompressedKeyLen(); keyWrapper.codec = this.in.getCompressionCodec(); valueWrapper.valueBuffer = this.in.getCurrentValueBufferObj(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/RCFileKeyBufferWrapper.java b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/RCFileKeyBufferWrapper.java index f7eacdc..f7b2645 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/RCFileKeyBufferWrapper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/RCFileKeyBufferWrapper.java @@ -30,9 +30,6 @@ public class RCFileKeyBufferWrapper implements WritableComparable { protected KeyBuffer keyBuffer; - protected int recordLength; - protected int keyLength; - protected int compressedKeyLength; protected CompressionCodec codec; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/RCFileMergeMapper.java b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/RCFileMergeMapper.java index bb1e3c9..cc480a1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/RCFileMergeMapper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/RCFileMergeMapper.java @@ -108,8 +108,7 @@ public class RCFileMergeMapper extends MapReduceBase implements "RCFileMerge failed because the input files use different CompressionCodec or have different column number setting."); } - outWriter.flushBlock(key.keyBuffer, value.valueBuffer, key.recordLength, - key.keyLength, key.compressedKeyLength); + outWriter.flushBlock(key.keyBuffer, value.valueBuffer, 0); } catch (Throwable e) { this.exception = true; close(); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java b/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java index 8bb6f3a..a9178e5 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java @@ -33,6 +33,8 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.io.RCFile.KeyBuffer; +import org.apache.hadoop.hive.ql.io.RCFile.ValueBuffer; import org.apache.hadoop.hive.serde.Constants; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.hive.serde2.SerDeException; @@ -46,8 +48,10 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; import org.apache.hadoop.hive.serde2.objectinspector.StructField; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption; +import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.compress.CompressionCodec; @@ -63,6 +67,11 @@ import org.apache.hadoop.mapred.Reporter; */ public class TestRCFile extends TestCase { + @Override + protected void setUp() throws Exception { + ColumnProjectionUtils.setFullyReadColumns(conf); + } + private static final Log LOG = LogFactory.getLog(TestRCFile.class); private static Configuration conf = new Configuration(); @@ -250,14 +259,15 @@ public class TestRCFile extends TestCase { LOG.info("file = " + file); TestRCFile test = new TestRCFile(); - // test.performanceTest(); - - test.testSimpleReadAndWrite(); - - test.writeTest(fs, count, file, bytesArray); - test.fullyReadTest(fs, count, file); - test.partialReadTest(fs, count, file); + //test.testSimpleReadAndWrite(); + //test.writeTest(fs, count, file, bytesArray); + //test.fullyReadTest(fs, count, file); + //test.partialReadTest(fs, count, file); + test.testReadWriteSEQUncompressed(); + test.testReadWriteSEQCompressed(); + test.testReadv6dot0Uncompressed(); + test.testReadv6dot0Compressed(); System.out.println("Finished."); } finally { fs.close(); @@ -387,6 +397,234 @@ public class TestRCFile extends TestCase { LOG.debug("reading fully costs:" + cost + " milliseconds"); } + public void testReadWriteSEQUncompressed() throws IOException, SerDeException { + + //write as RC + fs.delete(file, true); + + Text[][] records = { + {new Text("a"), new Text("1")}, + {new Text("bb"), new Text("22")}, + {new Text("a"), new Text("22")}, + {new Text("bb"), new Text("1")} + }; + int numRows = records.length; + int numCols = records[0].length; + + RCFileOutputFormat.setColumnNumber(conf, numCols); + RCFile.Writer writer = new RCFile.Writer(fs, conf, file); + BytesRefArrayWritable bytes = new BytesRefArrayWritable(numCols); + + for (int r = 0; r < numRows; ++r) { + for (int i = 0; i < records[r].length; i++) { + byte[] bytearray = records[r][i].getBytes(); + BytesRefWritable cu = new BytesRefWritable(bytearray, 0, + bytearray.length); + bytes.set(i, cu); + } + writer.append(bytes); + bytes.clear(); + } + writer.close(); + + //read as SEQ + SequenceFile.Reader reader = new SequenceFile.Reader(fs, file, conf); + + int numColumns = Integer.parseInt(reader.getMetadata().get( + new Text(RCFile.COLUMN_NUMBER_METADATA_STR)).toString()); + assertEquals(numCols, numColumns); + + KeyBuffer key = new KeyBuffer(RCFile.VERSION_WITH_METADATA, + Integer.parseInt( + reader.getMetadata().get(new Text(RCFile.RCFILE_VERSION_METADATA_STR)).toString()), + numColumns, null); + ValueBuffer value = new ValueBuffer(key); + reader.next(key, value); + reader.close(); + + fs.delete(file, true); + //write as SEQ + Path seq = new Path(file.getName() + ".seq"); + fs.delete(seq, true); + SequenceFile.Writer seqWriter = new SequenceFile.Writer(fs, conf, seq, KeyBuffer.class, + ValueBuffer.class, null, reader.getMetadata()); + seqWriter.append(key, value); + seqWriter.close(); + + //read as RC + RCFile.Reader rcreader = new RCFile.Reader(fs, seq, conf); + for (int i = 0; i < numRows; i++) { + LongWritable rowID = new LongWritable(); + rcreader.next(rowID); + BytesRefArrayWritable cols = new BytesRefArrayWritable(); + rcreader.getCurrentRow(cols); + cols.resetValid(numCols); + for (int j = 0; j < numCols; j++) { + byte[] expected = records[i][j].getBytes(); + byte[] actual = cols.get(j).getBytesCopy(); + assertEquals(expected.length, actual.length); + for (int k = 0; k < expected.length; ++k) { + assertEquals(expected[k], actual[k]); + } + } + } + rcreader.close(); + + fs.delete(seq, true); + } + + public void testReadWriteSEQCompressed() throws IOException, SerDeException { + + //write as RC + fs.delete(file, true); + + Text[][] records = { + {new Text("a"), new Text("1")}, + {new Text("bb"), new Text("22")}, + {new Text("a"), new Text("22")}, + {new Text("bb"), new Text("1")} + }; + int numRows = records.length; + int numCols = records[0].length; + + RCFileOutputFormat.setColumnNumber(conf, numCols); + DefaultCodec codec = new DefaultCodec(); + codec.setConf(conf); + + RCFile.Writer writer = new RCFile.Writer(fs, conf, file, null, codec); + BytesRefArrayWritable bytes = new BytesRefArrayWritable(numCols); + + for (int r = 0; r < numRows; ++r) { + for (int i = 0; i < records[r].length; i++) { + byte[] bytearray = records[r][i].getBytes(); + BytesRefWritable cu = new BytesRefWritable(bytearray, 0, + bytearray.length); + bytes.set(i, cu); + } + writer.append(bytes); + bytes.clear(); + } + writer.close(); + + //read as SEQ + SequenceFile.Reader reader = new SequenceFile.Reader(fs, file, conf); + + int numColumns = Integer.parseInt(reader.getMetadata().get( + new Text(RCFile.COLUMN_NUMBER_METADATA_STR)).toString()); + assertEquals(numCols, numColumns); + + KeyBuffer key = new KeyBuffer(RCFile.VERSION_WITH_METADATA, + Integer.parseInt( + reader.getMetadata().get(new Text(RCFile.RCFILE_VERSION_METADATA_STR)).toString()), + numColumns, codec); + ValueBuffer value = new ValueBuffer(key); + long pos = reader.getPosition(); + DataOutputBuffer tmp = new DataOutputBuffer(); + int currentKeyLength = reader.next(tmp); + int currentRecordLength = tmp.getLength(); + key.setRecordInfo(currentRecordLength, currentKeyLength); + reader.seek(pos); + reader.next(key, value); + reader.close(); + + //write as SEQ + Path seq = new Path(file.getName() + ".seq"); + fs.delete(seq, true); + SequenceFile.Writer seqWriter = new SequenceFile.Writer(fs, conf, seq, KeyBuffer.class, + ValueBuffer.class, null, reader.getMetadata()); + key.prepareToWrite(); + seqWriter.append(key, value); + seqWriter.close(); + + //read as RC + RCFile.Reader rcreader = new RCFile.Reader(fs, seq, conf); + for (int i = 0; i < numRows; i++) { + LongWritable rowID = new LongWritable(); + rcreader.next(rowID); + BytesRefArrayWritable cols = new BytesRefArrayWritable(); + rcreader.getCurrentRow(cols); + cols.resetValid(numCols); + for (int j = 0; j < numCols; j++) { + byte[] expected = records[i][j].getBytes(); + byte[] actual = cols.get(j).getBytesCopy(); + assertEquals(expected.length, actual.length); + for (int k = 0; k < expected.length; ++k) { + assertEquals(expected[k], actual[k]); + } + } + } + rcreader.close(); + + fs.delete(file, true); + fs.delete(seq, true); + } + + public void testReadv6dot0Uncompressed() throws IOException, SerDeException { + + Text[][] records = { + {new Text("a"), new Text("1")}, + {new Text("bb"), new Text("22")}, + {new Text("a"), new Text("22")}, + {new Text("bb"), new Text("1")} + }; + int numRows = records.length; + int numCols = records[0].length; + + Path dir = new Path(System.getProperty("test.src.data.dir", "data"), "files"); + + Path rcfile = new Path(dir, "test_v6_uncompressed.rc"); + RCFile.Reader rcreader = new RCFile.Reader(fs, rcfile, conf); + for (int i = 0; i < numRows; i++) { + LongWritable rowID = new LongWritable(); + rcreader.next(rowID); + BytesRefArrayWritable cols = new BytesRefArrayWritable(); + rcreader.getCurrentRow(cols); + cols.resetValid(numCols); + for (int j = 0; j < numCols; j++) { + byte[] expected = records[i][j].getBytes(); + byte[] actual = cols.get(j).getBytesCopy(); + assertEquals(expected.length, actual.length); + for (int k = 0; k < expected.length; ++k) { + assertEquals(expected[k], actual[k]); + } + } + } + rcreader.close(); + } + + public void testReadv6dot0Compressed() throws IOException, SerDeException { + + Text[][] records = { + {new Text("a"), new Text("1")}, + {new Text("bb"), new Text("22")}, + {new Text("a"), new Text("22")}, + {new Text("bb"), new Text("1")} + }; + int numRows = records.length; + int numCols = records[0].length; + + Path dir = new Path(System.getProperty("test.src.data.dir", "data"), "files"); + + Path rcfile = new Path(dir, "test_v6_compressed.rc"); + RCFile.Reader rcreader = new RCFile.Reader(fs, rcfile, conf); + for (int i = 0; i < numRows; i++) { + LongWritable rowID = new LongWritable(); + rcreader.next(rowID); + BytesRefArrayWritable cols = new BytesRefArrayWritable(); + rcreader.getCurrentRow(cols); + cols.resetValid(numCols); + for (int j = 0; j < numCols; j++) { + byte[] expected = records[i][j].getBytes(); + byte[] actual = cols.get(j).getBytesCopy(); + assertEquals(expected.length, actual.length); + for (int k = 0; k < expected.length; ++k) { + assertEquals(expected[k], actual[k]); + } + } + } + rcreader.close(); + } + public void testSynAndSplit() throws IOException { splitBeforeSync(); splitRightBeforeSync(); diff --git a/ql/src/test/results/clientpositive/alter_merge.q.out b/ql/src/test/results/clientpositive/alter_merge.q.out index 25f36c0..a4c1781 100644 --- a/ql/src/test/results/clientpositive/alter_merge.q.out +++ b/ql/src/test/results/clientpositive/alter_merge.q.out @@ -38,25 +38,25 @@ totalFileSize:636 maxFileSize:222 minFileSize:206 lastAccessTime:0 -lastUpdateTime:1300680902000 +lastUpdateTime:1301307548000 PREHOOK: query: select count(1) from src_rc_merge_test PREHOOK: type: QUERY PREHOOK: Input: default@src_rc_merge_test -PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-15-02_678_8520633473198390721/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-19-09_231_2198985670130000507/-mr-10000 POSTHOOK: query: select count(1) from src_rc_merge_test POSTHOOK: type: QUERY POSTHOOK: Input: default@src_rc_merge_test -POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-15-02_678_8520633473198390721/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-19-09_231_2198985670130000507/-mr-10000 15 PREHOOK: query: select sum(hash(key)), sum(hash(value)) from src_rc_merge_test PREHOOK: type: QUERY PREHOOK: Input: default@src_rc_merge_test -PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-15-10_671_3103325352622055634/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-19-29_311_966801859426201691/-mr-10000 POSTHOOK: query: select sum(hash(key)), sum(hash(value)) from src_rc_merge_test POSTHOOK: type: QUERY POSTHOOK: Input: default@src_rc_merge_test -POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-15-10_671_3103325352622055634/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-19-29_311_966801859426201691/-mr-10000 214 -7678496319 PREHOOK: query: alter table src_rc_merge_test concatenate PREHOOK: type: ALTER_TABLE_MERGE @@ -79,29 +79,29 @@ columns:struct columns { i32 key, string value} partitioned:false partitionColumns: totalNumberFiles:1 -totalFileSize:334 -maxFileSize:334 -minFileSize:334 +totalFileSize:365 +maxFileSize:365 +minFileSize:365 lastAccessTime:0 -lastUpdateTime:1300680922000 +lastUpdateTime:1301307580000 PREHOOK: query: select count(1) from src_rc_merge_test PREHOOK: type: QUERY PREHOOK: Input: default@src_rc_merge_test -PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-15-23_887_8242125214309556968/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-19-42_625_6930629717019904929/-mr-10000 POSTHOOK: query: select count(1) from src_rc_merge_test POSTHOOK: type: QUERY POSTHOOK: Input: default@src_rc_merge_test -POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-15-23_887_8242125214309556968/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-19-42_625_6930629717019904929/-mr-10000 15 PREHOOK: query: select sum(hash(key)), sum(hash(value)) from src_rc_merge_test PREHOOK: type: QUERY PREHOOK: Input: default@src_rc_merge_test -PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-15-31_244_4854539946976169821/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-19-58_205_5938290326044947890/-mr-10000 POSTHOOK: query: select sum(hash(key)), sum(hash(value)) from src_rc_merge_test POSTHOOK: type: QUERY POSTHOOK: Input: default@src_rc_merge_test -POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-15-31_244_4854539946976169821/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-19-58_205_5938290326044947890/-mr-10000 214 -7678496319 PREHOOK: query: create table src_rc_merge_test_part(key int, value string) partitioned by (ds string) stored as rcfile PREHOOK: type: CREATETABLE @@ -150,25 +150,25 @@ totalFileSize:636 maxFileSize:222 minFileSize:206 lastAccessTime:0 -lastUpdateTime:1300680946000 +lastUpdateTime:1301307608000 PREHOOK: query: select count(1) from src_rc_merge_test_part PREHOOK: type: QUERY PREHOOK: Input: default@src_rc_merge_test_part@ds=2011 -PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-15-47_140_6546800621174301781/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-20-09_216_4859589248335187490/-mr-10000 POSTHOOK: query: select count(1) from src_rc_merge_test_part POSTHOOK: type: QUERY POSTHOOK: Input: default@src_rc_merge_test_part@ds=2011 -POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-15-47_140_6546800621174301781/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-20-09_216_4859589248335187490/-mr-10000 15 PREHOOK: query: select sum(hash(key)), sum(hash(value)) from src_rc_merge_test_part PREHOOK: type: QUERY PREHOOK: Input: default@src_rc_merge_test_part@ds=2011 -PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-15-58_930_7643638670315828175/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-20-17_812_7453237252447967192/-mr-10000 POSTHOOK: query: select sum(hash(key)), sum(hash(value)) from src_rc_merge_test_part POSTHOOK: type: QUERY POSTHOOK: Input: default@src_rc_merge_test_part@ds=2011 -POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-15-58_930_7643638670315828175/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-20-17_812_7453237252447967192/-mr-10000 214 -7678496319 PREHOOK: query: alter table src_rc_merge_test_part partition (ds='2011') concatenate PREHOOK: type: ALTER_PARTITION_MERGE @@ -191,29 +191,29 @@ columns:struct columns { i32 key, string value} partitioned:true partitionColumns:struct partition_columns { string ds} totalNumberFiles:1 -totalFileSize:334 -maxFileSize:334 -minFileSize:334 +totalFileSize:365 +maxFileSize:365 +minFileSize:365 lastAccessTime:0 -lastUpdateTime:1300680970000 +lastUpdateTime:1301307633000 PREHOOK: query: select count(1) from src_rc_merge_test_part PREHOOK: type: QUERY PREHOOK: Input: default@src_rc_merge_test_part@ds=2011 -PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-16-11_200_6628817362773769157/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-20-34_091_154347906019109002/-mr-10000 POSTHOOK: query: select count(1) from src_rc_merge_test_part POSTHOOK: type: QUERY POSTHOOK: Input: default@src_rc_merge_test_part@ds=2011 -POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-16-11_200_6628817362773769157/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-20-34_091_154347906019109002/-mr-10000 15 PREHOOK: query: select sum(hash(key)), sum(hash(value)) from src_rc_merge_test_part PREHOOK: type: QUERY PREHOOK: Input: default@src_rc_merge_test_part@ds=2011 -PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-16-17_532_734781450927092021/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-20-41_781_3510087503600279899/-mr-10000 POSTHOOK: query: select sum(hash(key)), sum(hash(value)) from src_rc_merge_test_part POSTHOOK: type: QUERY POSTHOOK: Input: default@src_rc_merge_test_part@ds=2011 -POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-20_21-16-17_532_734781450927092021/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-20-41_781_3510087503600279899/-mr-10000 214 -7678496319 PREHOOK: query: drop table src_rc_merge_test PREHOOK: type: DROPTABLE diff --git a/ql/src/test/results/clientpositive/alter_merge_stats.q.out b/ql/src/test/results/clientpositive/alter_merge_stats.q.out index 243f7cc..7cb7021 100644 --- a/ql/src/test/results/clientpositive/alter_merge_stats.q.out +++ b/ql/src/test/results/clientpositive/alter_merge_stats.q.out @@ -38,7 +38,7 @@ totalFileSize:636 maxFileSize:222 minFileSize:206 lastAccessTime:0 -lastUpdateTime:1300680991000 +lastUpdateTime:1301307858000 PREHOOK: query: desc extended src_rc_merge_test_stat PREHOOK: type: DESCTABLE @@ -47,7 +47,7 @@ POSTHOOK: type: DESCTABLE key int from deserializer value string from deserializer -Detailed Table Information Table(tableName:src_rc_merge_test_stat, dbName:default, owner:krishnak, createTime:1300680988, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/src_rc_merge_test_stat, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[], parameters:{transient_lastDdlTime=1300680991}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:src_rc_merge_test_stat, dbName:default, owner:krishnak, createTime:1301307856, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/src_rc_merge_test_stat, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[], parameters:{transient_lastDdlTime=1301307858}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) PREHOOK: query: analyze table src_rc_merge_test_stat compute statistics PREHOOK: type: QUERY PREHOOK: Input: default@src_rc_merge_test_stat @@ -63,7 +63,7 @@ POSTHOOK: type: DESCTABLE key int from deserializer value string from deserializer -Detailed Table Information Table(tableName:src_rc_merge_test_stat, dbName:default, owner:krishnak, createTime:1300680988, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/src_rc_merge_test_stat, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[], parameters:{numPartitions=0, numFiles=3, transient_lastDdlTime=1300680999, numRows=6, totalSize=636}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:src_rc_merge_test_stat, dbName:default, owner:krishnak, createTime:1301307856, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/src_rc_merge_test_stat, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[], parameters:{numPartitions=0, numFiles=3, transient_lastDdlTime=1301307870, numRows=6, totalSize=636}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) PREHOOK: query: alter table src_rc_merge_test_stat concatenate PREHOOK: type: ALTER_TABLE_MERGE PREHOOK: Input: default@src_rc_merge_test_stat @@ -85,11 +85,11 @@ columns:struct columns { i32 key, string value} partitioned:false partitionColumns: totalNumberFiles:1 -totalFileSize:334 -maxFileSize:334 -minFileSize:334 +totalFileSize:365 +maxFileSize:365 +minFileSize:365 lastAccessTime:0 -lastUpdateTime:1300681001000 +lastUpdateTime:1301307873000 PREHOOK: query: desc extended src_rc_merge_test_stat PREHOOK: type: DESCTABLE @@ -98,7 +98,7 @@ POSTHOOK: type: DESCTABLE key int from deserializer value string from deserializer -Detailed Table Information Table(tableName:src_rc_merge_test_stat, dbName:default, owner:krishnak, createTime:1300680988, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/src_rc_merge_test_stat, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, transient_lastDdlTime=1300681002, numRows=6, totalSize=334}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:src_rc_merge_test_stat, dbName:default, owner:krishnak, createTime:1301307856, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/src_rc_merge_test_stat, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, transient_lastDdlTime=1301307874, numRows=6, totalSize=365}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) PREHOOK: query: create table src_rc_merge_test_part_stat(key int, value string) partitioned by (ds string) stored as rcfile PREHOOK: type: CREATETABLE POSTHOOK: query: create table src_rc_merge_test_part_stat(key int, value string) partitioned by (ds string) stored as rcfile @@ -146,7 +146,7 @@ totalFileSize:636 maxFileSize:222 minFileSize:206 lastAccessTime:0 -lastUpdateTime:1300681004000 +lastUpdateTime:1301307877000 PREHOOK: query: desc extended src_rc_merge_test_part_stat PREHOOK: type: DESCTABLE @@ -156,7 +156,7 @@ key int from deserializer value string from deserializer ds string -Detailed Table Information Table(tableName:src_rc_merge_test_part_stat, dbName:default, owner:krishnak, createTime:1300681002, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/src_rc_merge_test_part_stat, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1300681002}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:src_rc_merge_test_part_stat, dbName:default, owner:krishnak, createTime:1301307875, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/src_rc_merge_test_part_stat, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1301307875}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) PREHOOK: query: analyze table src_rc_merge_test_part_stat partition(ds='2011') compute statistics PREHOOK: type: QUERY PREHOOK: Input: default@src_rc_merge_test_part_stat@ds=2011 @@ -175,7 +175,7 @@ key int from deserializer value string from deserializer ds string -Detailed Table Information Table(tableName:src_rc_merge_test_part_stat, dbName:default, owner:krishnak, createTime:1300681002, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/src_rc_merge_test_part_stat, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{numPartitions=1, numFiles=3, transient_lastDdlTime=1300681013, numRows=6, totalSize=636}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:src_rc_merge_test_part_stat, dbName:default, owner:krishnak, createTime:1301307875, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/src_rc_merge_test_part_stat, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{numPartitions=1, numFiles=3, transient_lastDdlTime=1301307885, numRows=6, totalSize=636}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) PREHOOK: query: alter table src_rc_merge_test_part_stat partition (ds='2011') concatenate PREHOOK: type: ALTER_PARTITION_MERGE PREHOOK: Input: default@src_rc_merge_test_part_stat @@ -197,11 +197,11 @@ columns:struct columns { i32 key, string value} partitioned:true partitionColumns:struct partition_columns { string ds} totalNumberFiles:1 -totalFileSize:334 -maxFileSize:334 -minFileSize:334 +totalFileSize:365 +maxFileSize:365 +minFileSize:365 lastAccessTime:0 -lastUpdateTime:1300681015000 +lastUpdateTime:1301307888000 PREHOOK: query: desc extended src_rc_merge_test_part_stat PREHOOK: type: DESCTABLE @@ -211,7 +211,7 @@ key int from deserializer value string from deserializer ds string -Detailed Table Information Table(tableName:src_rc_merge_test_part_stat, dbName:default, owner:krishnak, createTime:1300681002, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/src_rc_merge_test_part_stat, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, transient_lastDdlTime=1300681015, numRows=6, totalSize=334}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:src_rc_merge_test_part_stat, dbName:default, owner:krishnak, createTime:1301307875, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/src_rc_merge_test_part_stat, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, transient_lastDdlTime=1301307888, numRows=6, totalSize=365}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) PREHOOK: query: drop table src_rc_merge_test_stat PREHOOK: type: DROPTABLE PREHOOK: Input: default@src_rc_merge_test_stat diff --git a/ql/src/test/results/clientpositive/partition_wise_fileformat.q.out b/ql/src/test/results/clientpositive/partition_wise_fileformat.q.out index cee2e72..d4050ce 100644 --- a/ql/src/test/results/clientpositive/partition_wise_fileformat.q.out +++ b/ql/src/test/results/clientpositive/partition_wise_fileformat.q.out @@ -20,8 +20,8 @@ POSTHOOK: type: SHOW_TABLESTATUS POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] tableName:partition_test_partitioned -owner:njain -location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/partition_test_partitioned +owner:krishnak +location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/partition_test_partitioned inputformat:org.apache.hadoop.mapred.TextInputFormat outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat columns:struct columns { string key, string value} @@ -32,7 +32,7 @@ totalFileSize:216 maxFileSize:216 minFileSize:216 lastAccessTime:0 -lastUpdateTime:1282030124000 +lastUpdateTime:1301308029000 PREHOOK: query: show table extended like partition_test_partitioned partition(dt=100) PREHOOK: type: SHOW_TABLESTATUS @@ -41,8 +41,8 @@ POSTHOOK: type: SHOW_TABLESTATUS POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] tableName:partition_test_partitioned -owner:njain -location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/partition_test_partitioned/dt=100 +owner:krishnak +location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/partition_test_partitioned/dt=100 inputformat:org.apache.hadoop.mapred.TextInputFormat outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat columns:struct columns { string key, string value} @@ -53,16 +53,16 @@ totalFileSize:216 maxFileSize:216 minFileSize:216 lastAccessTime:0 -lastUpdateTime:1282030124000 +lastUpdateTime:1301308029000 PREHOOK: query: select key from partition_test_partitioned where dt=100 PREHOOK: type: QUERY PREHOOK: Input: default@partition_test_partitioned@dt=100 -PREHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-28-45_751_169876228483688017/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-27-10_534_3517294960259884011/-mr-10000 POSTHOOK: query: select key from partition_test_partitioned where dt=100 POSTHOOK: type: QUERY POSTHOOK: Input: default@partition_test_partitioned@dt=100 -POSTHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-28-45_751_169876228483688017/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-27-10_534_3517294960259884011/-mr-10000 POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] 238 @@ -93,11 +93,11 @@ POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(s PREHOOK: query: select key from partition_test_partitioned PREHOOK: type: QUERY PREHOOK: Input: default@partition_test_partitioned@dt=100 -PREHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-28-49_132_7250201183436014876/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-27-17_139_7616467668164915031/-mr-10000 POSTHOOK: query: select key from partition_test_partitioned POSTHOOK: type: QUERY POSTHOOK: Input: default@partition_test_partitioned@dt=100 -POSTHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-28-49_132_7250201183436014876/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-27-17_139_7616467668164915031/-mr-10000 POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] 238 @@ -156,19 +156,19 @@ POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(s POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] tableName:partition_test_partitioned -owner:njain -location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/partition_test_partitioned +owner:krishnak +location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/partition_test_partitioned inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat columns:struct columns { string key, string value} partitioned:true partitionColumns:struct partition_columns { string dt} totalNumberFiles:2 -totalFileSize:586 -maxFileSize:370 +totalFileSize:617 +maxFileSize:401 minFileSize:216 lastAccessTime:0 -lastUpdateTime:1282030136000 +lastUpdateTime:1301308055000 PREHOOK: query: show table extended like partition_test_partitioned partition(dt=100) PREHOOK: type: SHOW_TABLESTATUS @@ -179,8 +179,8 @@ POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(s POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] tableName:partition_test_partitioned -owner:njain -location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/partition_test_partitioned/dt=100 +owner:krishnak +location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/partition_test_partitioned/dt=100 inputformat:org.apache.hadoop.mapred.TextInputFormat outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat columns:struct columns { string key, string value} @@ -191,7 +191,7 @@ totalFileSize:216 maxFileSize:216 minFileSize:216 lastAccessTime:0 -lastUpdateTime:1282030136000 +lastUpdateTime:1301308055000 PREHOOK: query: show table extended like partition_test_partitioned partition(dt=101) PREHOOK: type: SHOW_TABLESTATUS @@ -202,28 +202,28 @@ POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(s POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] tableName:partition_test_partitioned -owner:njain -location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/partition_test_partitioned/dt=101 +owner:krishnak +location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/partition_test_partitioned/dt=101 inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat columns:struct columns { string key, string value} partitioned:true partitionColumns:struct partition_columns { string dt} totalNumberFiles:1 -totalFileSize:370 -maxFileSize:370 -minFileSize:370 +totalFileSize:401 +maxFileSize:401 +minFileSize:401 lastAccessTime:0 -lastUpdateTime:1282030136000 +lastUpdateTime:1301308055000 PREHOOK: query: select key from partition_test_partitioned where dt=100 PREHOOK: type: QUERY PREHOOK: Input: default@partition_test_partitioned@dt=100 -PREHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-28-57_738_4221917279395084924/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-27-37_639_5228682805283771837/-mr-10000 POSTHOOK: query: select key from partition_test_partitioned where dt=100 POSTHOOK: type: QUERY POSTHOOK: Input: default@partition_test_partitioned@dt=100 -POSTHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-28-57_738_4221917279395084924/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-27-37_639_5228682805283771837/-mr-10000 POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] @@ -256,11 +256,11 @@ POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).value SIMPLE [(s PREHOOK: query: select key from partition_test_partitioned where dt=101 PREHOOK: type: QUERY PREHOOK: Input: default@partition_test_partitioned@dt=101 -PREHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-01_104_7633460107807770575/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-27-44_884_7494292850124737954/-mr-10000 POSTHOOK: query: select key from partition_test_partitioned where dt=101 POSTHOOK: type: QUERY POSTHOOK: Input: default@partition_test_partitioned@dt=101 -POSTHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-01_104_7633460107807770575/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-27-44_884_7494292850124737954/-mr-10000 POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] @@ -294,12 +294,12 @@ PREHOOK: query: select key from partition_test_partitioned PREHOOK: type: QUERY PREHOOK: Input: default@partition_test_partitioned@dt=100 PREHOOK: Input: default@partition_test_partitioned@dt=101 -PREHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-04_462_387937014678398458/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-27-52_961_222322694600111869/-mr-10000 POSTHOOK: query: select key from partition_test_partitioned POSTHOOK: type: QUERY POSTHOOK: Input: default@partition_test_partitioned@dt=100 POSTHOOK: Input: default@partition_test_partitioned@dt=101 -POSTHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-04_462_387937014678398458/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-27-52_961_222322694600111869/-mr-10000 POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] @@ -391,19 +391,19 @@ POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).value SIMPLE [(s POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] tableName:partition_test_partitioned -owner:njain -location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/partition_test_partitioned +owner:krishnak +location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/partition_test_partitioned inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat columns:struct columns { string key, string value} partitioned:true partitionColumns:struct partition_columns { string dt} totalNumberFiles:3 -totalFileSize:1474 +totalFileSize:1505 maxFileSize:888 minFileSize:216 lastAccessTime:0 -lastUpdateTime:1282030151000 +lastUpdateTime:1301308092000 PREHOOK: query: show table extended like partition_test_partitioned partition(dt=100) PREHOOK: type: SHOW_TABLESTATUS @@ -416,8 +416,8 @@ POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).value SIMPLE [(s POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] tableName:partition_test_partitioned -owner:njain -location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/partition_test_partitioned/dt=100 +owner:krishnak +location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/partition_test_partitioned/dt=100 inputformat:org.apache.hadoop.mapred.TextInputFormat outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat columns:struct columns { string key, string value} @@ -428,7 +428,7 @@ totalFileSize:216 maxFileSize:216 minFileSize:216 lastAccessTime:0 -lastUpdateTime:1282030151000 +lastUpdateTime:1301308092000 PREHOOK: query: show table extended like partition_test_partitioned partition(dt=101) PREHOOK: type: SHOW_TABLESTATUS @@ -441,19 +441,19 @@ POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).value SIMPLE [(s POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] tableName:partition_test_partitioned -owner:njain -location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/partition_test_partitioned/dt=101 +owner:krishnak +location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/partition_test_partitioned/dt=101 inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat columns:struct columns { string key, string value} partitioned:true partitionColumns:struct partition_columns { string dt} totalNumberFiles:1 -totalFileSize:370 -maxFileSize:370 -minFileSize:370 +totalFileSize:401 +maxFileSize:401 +minFileSize:401 lastAccessTime:0 -lastUpdateTime:1282030151000 +lastUpdateTime:1301308092000 PREHOOK: query: show table extended like partition_test_partitioned partition(dt=102) PREHOOK: type: SHOW_TABLESTATUS @@ -466,8 +466,8 @@ POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).value SIMPLE [(s POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] tableName:partition_test_partitioned -owner:njain -location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/partition_test_partitioned/dt=102 +owner:krishnak +location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/partition_test_partitioned/dt=102 inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat columns:struct columns { string key, string value} @@ -478,16 +478,16 @@ totalFileSize:888 maxFileSize:888 minFileSize:888 lastAccessTime:0 -lastUpdateTime:1282030151000 +lastUpdateTime:1301308092000 PREHOOK: query: select key from partition_test_partitioned where dt=100 PREHOOK: type: QUERY PREHOOK: Input: default@partition_test_partitioned@dt=100 -PREHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-13_648_4852070201967333188/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-28-15_448_4730227192199108726/-mr-10000 POSTHOOK: query: select key from partition_test_partitioned where dt=100 POSTHOOK: type: QUERY POSTHOOK: Input: default@partition_test_partitioned@dt=100 -POSTHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-13_648_4852070201967333188/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-28-15_448_4730227192199108726/-mr-10000 POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] @@ -522,11 +522,11 @@ POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).value SIMPLE [(s PREHOOK: query: select key from partition_test_partitioned where dt=101 PREHOOK: type: QUERY PREHOOK: Input: default@partition_test_partitioned@dt=101 -PREHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-16_874_2144423925129213549/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-28-23_374_3223676181616394812/-mr-10000 POSTHOOK: query: select key from partition_test_partitioned where dt=101 POSTHOOK: type: QUERY POSTHOOK: Input: default@partition_test_partitioned@dt=101 -POSTHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-16_874_2144423925129213549/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-28-23_374_3223676181616394812/-mr-10000 POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] @@ -561,11 +561,11 @@ POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).value SIMPLE [(s PREHOOK: query: select key from partition_test_partitioned where dt=102 PREHOOK: type: QUERY PREHOOK: Input: default@partition_test_partitioned@dt=102 -PREHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-20_202_7935828228211072259/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-28-31_651_7624461423093868710/-mr-10000 POSTHOOK: query: select key from partition_test_partitioned where dt=102 POSTHOOK: type: QUERY POSTHOOK: Input: default@partition_test_partitioned@dt=102 -POSTHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-20_202_7935828228211072259/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-28-31_651_7624461423093868710/-mr-10000 POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] @@ -602,13 +602,13 @@ PREHOOK: type: QUERY PREHOOK: Input: default@partition_test_partitioned@dt=100 PREHOOK: Input: default@partition_test_partitioned@dt=101 PREHOOK: Input: default@partition_test_partitioned@dt=102 -PREHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-23_498_1472831792275217567/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-28-40_607_8576436476371286922/-mr-10000 POSTHOOK: query: select key from partition_test_partitioned POSTHOOK: type: QUERY POSTHOOK: Input: default@partition_test_partitioned@dt=100 POSTHOOK: Input: default@partition_test_partitioned@dt=101 POSTHOOK: Input: default@partition_test_partitioned@dt=102 -POSTHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-23_498_1472831792275217567/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-28-40_607_8576436476371286922/-mr-10000 POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] @@ -695,13 +695,13 @@ PREHOOK: type: QUERY PREHOOK: Input: default@partition_test_partitioned@dt=100 PREHOOK: Input: default@partition_test_partitioned@dt=101 PREHOOK: Input: default@partition_test_partitioned@dt=102 -PREHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-26_942_5352511744427811614/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-28-53_002_7001863270329093164/-mr-10000 POSTHOOK: query: select key from partition_test_partitioned where dt >=100 and dt <= 102 POSTHOOK: type: QUERY POSTHOOK: Input: default@partition_test_partitioned@dt=100 POSTHOOK: Input: default@partition_test_partitioned@dt=101 POSTHOOK: Input: default@partition_test_partitioned@dt=102 -POSTHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-26_942_5352511744427811614/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-28-53_002_7001863270329093164/-mr-10000 POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=100).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] diff --git a/ql/src/test/results/clientpositive/partition_wise_fileformat3.q.out b/ql/src/test/results/clientpositive/partition_wise_fileformat3.q.out index 067ab43..e6d25f2 100644 --- a/ql/src/test/results/clientpositive/partition_wise_fileformat3.q.out +++ b/ql/src/test/results/clientpositive/partition_wise_fileformat3.q.out @@ -28,19 +28,19 @@ POSTHOOK: type: SHOW_TABLESTATUS POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] tableName:partition_test_partitioned -owner:njain -location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/partition_test_partitioned/dt=101 +owner:krishnak +location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/partition_test_partitioned/dt=101 inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat columns:struct columns { string key, string value} partitioned:true partitionColumns:struct partition_columns { string dt} totalNumberFiles:1 -totalFileSize:370 -maxFileSize:370 -minFileSize:370 +totalFileSize:401 +maxFileSize:401 +minFileSize:401 lastAccessTime:0 -lastUpdateTime:1282030189000 +lastUpdateTime:1301308334000 PREHOOK: query: alter table partition_test_partitioned set fileformat Sequencefile PREHOOK: type: ALTERTABLE_FILEFORMAT @@ -73,8 +73,8 @@ POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).value SIMPLE [(s POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] tableName:partition_test_partitioned -owner:njain -location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/partition_test_partitioned/dt=102 +owner:krishnak +location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/partition_test_partitioned/dt=102 inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat columns:struct columns { string key, string value} @@ -85,16 +85,16 @@ totalFileSize:888 maxFileSize:888 minFileSize:888 lastAccessTime:0 -lastUpdateTime:1282030193000 +lastUpdateTime:1301308344000 PREHOOK: query: select key from partition_test_partitioned where dt=102 PREHOOK: type: QUERY PREHOOK: Input: default@partition_test_partitioned@dt=102 -PREHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-54_035_3465387967141592899/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-32-24_969_2312628843276952894/-mr-10000 POSTHOOK: query: select key from partition_test_partitioned where dt=102 POSTHOOK: type: QUERY POSTHOOK: Input: default@partition_test_partitioned@dt=102 -POSTHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-29-54_035_3465387967141592899/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-32-24_969_2312628843276952894/-mr-10000 POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] @@ -149,8 +149,8 @@ POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).value SIMPLE [(s POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=102).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] tableName:partition_test_partitioned -owner:njain -location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/partition_test_partitioned/dt=101 +owner:krishnak +location:pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/partition_test_partitioned/dt=101 inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat columns:struct columns { string key, string value} @@ -161,16 +161,16 @@ totalFileSize:888 maxFileSize:888 minFileSize:888 lastAccessTime:0 -lastUpdateTime:1282030201000 +lastUpdateTime:1301308361000 PREHOOK: query: select key from partition_test_partitioned where dt=101 PREHOOK: type: QUERY PREHOOK: Input: default@partition_test_partitioned@dt=101 -PREHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-30-01_709_7422795570962759074/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-32-42_031_6718146757498865551/-mr-10000 POSTHOOK: query: select key from partition_test_partitioned where dt=101 POSTHOOK: type: QUERY POSTHOOK: Input: default@partition_test_partitioned@dt=101 -POSTHOOK: Output: file:/tmp/njain/hive_2010-08-17_00-30-01_709_7422795570962759074/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-32-42_031_6718146757498865551/-mr-10000 POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: partition_test_partitioned PARTITION(dt=101).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] diff --git a/ql/src/test/results/clientpositive/sample10.q.out b/ql/src/test/results/clientpositive/sample10.q.out index 50406c3..ac2c3a8 100644 --- a/ql/src/test/results/clientpositive/sample10.q.out +++ b/ql/src/test/results/clientpositive/sample10.q.out @@ -104,12 +104,12 @@ STAGE PLANS: type: bigint Needs Tagging: false Path -> Alias: - pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-08/hr=11/000000_0 [srcpartbucket] - pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-08/hr=12/000000_0 [srcpartbucket] - pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-09/hr=11/000000_0 [srcpartbucket] - pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-09/hr=12/000000_0 [srcpartbucket] + pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-08/hr=11/000000_0 [srcpartbucket] + pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-08/hr=12/000000_0 [srcpartbucket] + pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-09/hr=11/000000_0 [srcpartbucket] + pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-09/hr=12/000000_0 [srcpartbucket] Path -> Partition: - pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-08/hr=11/000000_0 + pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-08/hr=11/000000_0 Partition base file name: 000000_0 input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat @@ -124,7 +124,7 @@ STAGE PLANS: columns.types string:string file.inputformat org.apache.hadoop.hive.ql.io.RCFileInputFormat file.outputformat org.apache.hadoop.hive.ql.io.RCFileOutputFormat - location pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-08/hr=11 + location pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-08/hr=11 name default.srcpartbucket numFiles 16 numPartitions 4 @@ -133,8 +133,8 @@ STAGE PLANS: serialization.ddl struct srcpartbucket { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - totalSize 2748 - transient_lastDdlTime 1297386150 + totalSize 3244 + transient_lastDdlTime 1301308525 serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat @@ -146,7 +146,7 @@ STAGE PLANS: columns.types string:string file.inputformat org.apache.hadoop.hive.ql.io.RCFileInputFormat file.outputformat org.apache.hadoop.hive.ql.io.RCFileOutputFormat - location pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket + location pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket name default.srcpartbucket numFiles 16 numPartitions 4 @@ -155,12 +155,12 @@ STAGE PLANS: serialization.ddl struct srcpartbucket { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - totalSize 2748 - transient_lastDdlTime 1297386150 + totalSize 3244 + transient_lastDdlTime 1301308525 serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe name: default.srcpartbucket name: default.srcpartbucket - pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-08/hr=12/000000_0 + pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-08/hr=12/000000_0 Partition base file name: 000000_0 input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat @@ -175,7 +175,7 @@ STAGE PLANS: columns.types string:string file.inputformat org.apache.hadoop.hive.ql.io.RCFileInputFormat file.outputformat org.apache.hadoop.hive.ql.io.RCFileOutputFormat - location pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-08/hr=12 + location pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-08/hr=12 name default.srcpartbucket numFiles 16 numPartitions 4 @@ -184,8 +184,8 @@ STAGE PLANS: serialization.ddl struct srcpartbucket { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - totalSize 2748 - transient_lastDdlTime 1297386150 + totalSize 3244 + transient_lastDdlTime 1301308525 serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat @@ -197,7 +197,7 @@ STAGE PLANS: columns.types string:string file.inputformat org.apache.hadoop.hive.ql.io.RCFileInputFormat file.outputformat org.apache.hadoop.hive.ql.io.RCFileOutputFormat - location pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket + location pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket name default.srcpartbucket numFiles 16 numPartitions 4 @@ -206,12 +206,12 @@ STAGE PLANS: serialization.ddl struct srcpartbucket { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - totalSize 2748 - transient_lastDdlTime 1297386150 + totalSize 3244 + transient_lastDdlTime 1301308525 serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe name: default.srcpartbucket name: default.srcpartbucket - pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-09/hr=11/000000_0 + pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-09/hr=11/000000_0 Partition base file name: 000000_0 input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat @@ -226,7 +226,7 @@ STAGE PLANS: columns.types string:string file.inputformat org.apache.hadoop.hive.ql.io.RCFileInputFormat file.outputformat org.apache.hadoop.hive.ql.io.RCFileOutputFormat - location pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-09/hr=11 + location pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-09/hr=11 name default.srcpartbucket numFiles 16 numPartitions 4 @@ -235,8 +235,8 @@ STAGE PLANS: serialization.ddl struct srcpartbucket { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - totalSize 2748 - transient_lastDdlTime 1297386150 + totalSize 3244 + transient_lastDdlTime 1301308525 serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat @@ -248,7 +248,7 @@ STAGE PLANS: columns.types string:string file.inputformat org.apache.hadoop.hive.ql.io.RCFileInputFormat file.outputformat org.apache.hadoop.hive.ql.io.RCFileOutputFormat - location pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket + location pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket name default.srcpartbucket numFiles 16 numPartitions 4 @@ -257,12 +257,12 @@ STAGE PLANS: serialization.ddl struct srcpartbucket { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - totalSize 2748 - transient_lastDdlTime 1297386150 + totalSize 3244 + transient_lastDdlTime 1301308525 serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe name: default.srcpartbucket name: default.srcpartbucket - pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-09/hr=12/000000_0 + pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-09/hr=12/000000_0 Partition base file name: 000000_0 input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat @@ -277,7 +277,7 @@ STAGE PLANS: columns.types string:string file.inputformat org.apache.hadoop.hive.ql.io.RCFileInputFormat file.outputformat org.apache.hadoop.hive.ql.io.RCFileOutputFormat - location pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-09/hr=12 + location pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket/ds=2008-04-09/hr=12 name default.srcpartbucket numFiles 16 numPartitions 4 @@ -286,8 +286,8 @@ STAGE PLANS: serialization.ddl struct srcpartbucket { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - totalSize 2748 - transient_lastDdlTime 1297386150 + totalSize 3244 + transient_lastDdlTime 1301308525 serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat @@ -299,7 +299,7 @@ STAGE PLANS: columns.types string:string file.inputformat org.apache.hadoop.hive.ql.io.RCFileInputFormat file.outputformat org.apache.hadoop.hive.ql.io.RCFileOutputFormat - location pfile:/data/users/sdong/www/open-source-hive1/build/ql/test/data/warehouse/srcpartbucket + location pfile:/Users/krishnak/Projects/hdp/sources/hive-git-apache/build/ql/test/data/warehouse/srcpartbucket name default.srcpartbucket numFiles 16 numPartitions 4 @@ -308,8 +308,8 @@ STAGE PLANS: serialization.ddl struct srcpartbucket { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - totalSize 2748 - transient_lastDdlTime 1297386150 + totalSize 3244 + transient_lastDdlTime 1301308525 serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe name: default.srcpartbucket name: default.srcpartbucket @@ -333,9 +333,9 @@ STAGE PLANS: File Output Operator compressed: false GlobalTableId: 0 - directory: file:/tmp/sdong/hive_2011-02-10_17-02-30_127_2856183230982405624/-ext-10001 + directory: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-35-25_778_4151232483661284346/-ext-10001 NumFilesPerFileSink: 1 - Stats Publishing Key Prefix: file:/tmp/sdong/hive_2011-02-10_17-02-30_127_2856183230982405624/-ext-10001/ + Stats Publishing Key Prefix: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-35-25_778_4151232483661284346/-ext-10001/ table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -358,14 +358,14 @@ PREHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=11 PREHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=12 -PREHOOK: Output: file:/tmp/sdong/hive_2011-02-10_17-02-31_052_5780136652498214851/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-35-26_823_381668417196391502/-mr-10000 POSTHOOK: query: select ds, count(1) from srcpartbucket tablesample (bucket 1 out of 4 on key) where ds is not null group by ds POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=11 POSTHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=12 POSTHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=11 POSTHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=12 -POSTHOOK: Output: file:/tmp/sdong/hive_2011-02-10_17-02-31_052_5780136652498214851/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-35-26_823_381668417196391502/-mr-10000 POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-08,hr=12).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] @@ -382,14 +382,14 @@ PREHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=11 PREHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=12 -PREHOOK: Output: file:/tmp/sdong/hive_2011-02-10_17-02-35_963_7688294481780262172/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-35-36_537_2262690193138337942/-mr-10000 POSTHOOK: query: select ds, count(1) from srcpartbucket tablesample (bucket 1 out of 2 on key) where ds is not null group by ds POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=11 POSTHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=12 POSTHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=11 POSTHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=12 -POSTHOOK: Output: file:/tmp/sdong/hive_2011-02-10_17-02-35_963_7688294481780262172/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-35-36_537_2262690193138337942/-mr-10000 POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-08,hr=12).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] @@ -406,14 +406,14 @@ PREHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=11 PREHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=12 -PREHOOK: Output: file:/tmp/sdong/hive_2011-02-10_17-02-42_219_2445297351151986459/-mr-10000 +PREHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-35-50_075_1096572952564273976/-mr-10000 POSTHOOK: query: select * from srcpartbucket where ds is not null POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=11 POSTHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=12 POSTHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=11 POSTHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=12 -POSTHOOK: Output: file:/tmp/sdong/hive_2011-02-10_17-02-42_219_2445297351151986459/-mr-10000 +POSTHOOK: Output: file:/var/folders/67/67R3POPtF90VG63KSmCbcU++F0U/-Tmp-/krishnak/hive_2011-03-28_03-35-50_075_1096572952564273976/-mr-10000 POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-08,hr=12).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]