Return-Path: Delivered-To: apmail-hive-commits-archive@www.apache.org Received: (qmail 4474 invoked from network); 23 Feb 2011 23:59:05 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 23 Feb 2011 23:59:05 -0000 Received: (qmail 2715 invoked by uid 500); 23 Feb 2011 23:59:05 -0000 Delivered-To: apmail-hive-commits-archive@hive.apache.org Received: (qmail 2660 invoked by uid 500); 23 Feb 2011 23:59:05 -0000 Mailing-List: contact commits-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hive-dev@hive.apache.org Delivered-To: mailing list commits@hive.apache.org Received: (qmail 2652 invoked by uid 99); 23 Feb 2011 23:59:05 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 23 Feb 2011 23:59:04 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 23 Feb 2011 23:59:03 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 67AE223889EA; Wed, 23 Feb 2011 23:58:43 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1073992 [3/3] - in /hive/trunk: ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/io/ ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/ ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/ ql/src/java/org/a... Date: Wed, 23 Feb 2011 23:58:42 -0000 To: commits@hive.apache.org From: nzhang@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20110223235843.67AE223889EA@eris.apache.org> Added: hive/trunk/ql/src/test/results/clientpositive/alter_merge_stats.q.out URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/alter_merge_stats.q.out?rev=1073992&view=auto ============================================================================== --- hive/trunk/ql/src/test/results/clientpositive/alter_merge_stats.q.out (added) +++ hive/trunk/ql/src/test/results/clientpositive/alter_merge_stats.q.out Wed Feb 23 23:58:41 2011 @@ -0,0 +1,224 @@ +PREHOOK: query: create table src_rc_merge_test_stat(key int, value string) stored as rcfile +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table src_rc_merge_test_stat(key int, value string) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@src_rc_merge_test_stat +PREHOOK: query: load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_stat +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_stat +POSTHOOK: type: LOAD +POSTHOOK: Output: default@src_rc_merge_test_stat +PREHOOK: query: load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_stat +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_stat +POSTHOOK: type: LOAD +POSTHOOK: Output: default@src_rc_merge_test_stat +PREHOOK: query: load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_stat +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_stat +POSTHOOK: type: LOAD +POSTHOOK: Output: default@src_rc_merge_test_stat +PREHOOK: query: show table extended like `src_rc_merge_test_stat` +PREHOOK: type: SHOW_TABLESTATUS +POSTHOOK: query: show table extended like `src_rc_merge_test_stat` +POSTHOOK: type: SHOW_TABLESTATUS +tableName:src_rc_merge_test_stat +owner:heyongqiang +location:pfile:/Users/heyongqiang/Documents/workspace/Hive-3/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 +columns:struct columns { i32 key, string value} +partitioned:false +partitionColumns: +totalNumberFiles:3 +totalFileSize:636 +maxFileSize:222 +minFileSize:206 +lastAccessTime:0 +lastUpdateTime:1297289991000 + +PREHOOK: query: desc extended src_rc_merge_test_stat +PREHOOK: type: DESCTABLE +POSTHOOK: query: desc extended src_rc_merge_test_stat +POSTHOOK: type: DESCTABLE +key int from deserializer +value string from deserializer + +Detailed Table Information Table(tableName:src_rc_merge_test_stat, dbName:default, owner:heyongqiang, createTime:1297289989, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/heyongqiang/Documents/workspace/Hive-3/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=1297289991}, 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 +PREHOOK: Output: default@src_rc_merge_test_stat +POSTHOOK: query: analyze table src_rc_merge_test_stat compute statistics +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_rc_merge_test_stat +POSTHOOK: Output: default@src_rc_merge_test_stat +PREHOOK: query: desc extended src_rc_merge_test_stat +PREHOOK: type: DESCTABLE +POSTHOOK: query: desc extended src_rc_merge_test_stat +POSTHOOK: type: DESCTABLE +key int from deserializer +value string from deserializer + +Detailed Table Information Table(tableName:src_rc_merge_test_stat, dbName:default, owner:heyongqiang, createTime:1297289989, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/heyongqiang/Documents/workspace/Hive-3/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=1297289998, 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 +PREHOOK: Output: default@src_rc_merge_test_stat +POSTHOOK: query: alter table src_rc_merge_test_stat concatenate +POSTHOOK: type: ALTER_TABLE_MERGE +POSTHOOK: Input: default@src_rc_merge_test_stat +POSTHOOK: Output: default@src_rc_merge_test_stat +PREHOOK: query: show table extended like `src_rc_merge_test_stat` +PREHOOK: type: SHOW_TABLESTATUS +POSTHOOK: query: show table extended like `src_rc_merge_test_stat` +POSTHOOK: type: SHOW_TABLESTATUS +tableName:src_rc_merge_test_stat +owner:heyongqiang +location:pfile:/Users/heyongqiang/Documents/workspace/Hive-3/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 +columns:struct columns { i32 key, string value} +partitioned:false +partitionColumns: +totalNumberFiles:1 +totalFileSize:334 +maxFileSize:334 +minFileSize:334 +lastAccessTime:0 +lastUpdateTime:1297290000000 + +PREHOOK: query: desc extended src_rc_merge_test_stat +PREHOOK: type: DESCTABLE +POSTHOOK: query: desc extended src_rc_merge_test_stat +POSTHOOK: type: DESCTABLE +key int from deserializer +value string from deserializer + +Detailed Table Information Table(tableName:src_rc_merge_test_stat, dbName:default, owner:heyongqiang, createTime:1297289989, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/heyongqiang/Documents/workspace/Hive-3/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=1297290000, numRows=6, totalSize=334}, 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 +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@src_rc_merge_test_part_stat +PREHOOK: query: alter table src_rc_merge_test_part_stat add partition (ds='2011') +PREHOOK: type: ALTERTABLE_ADDPARTS +PREHOOK: Input: default@src_rc_merge_test_part_stat +POSTHOOK: query: alter table src_rc_merge_test_part_stat add partition (ds='2011') +POSTHOOK: type: ALTERTABLE_ADDPARTS +POSTHOOK: Input: default@src_rc_merge_test_part_stat +POSTHOOK: Output: default@src_rc_merge_test_part_stat@ds=2011 +PREHOOK: query: load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_part_stat partition (ds='2011') +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_part_stat partition (ds='2011') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@src_rc_merge_test_part_stat@ds=2011 +PREHOOK: query: load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_part_stat partition (ds='2011') +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_part_stat partition (ds='2011') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@src_rc_merge_test_part_stat@ds=2011 +PREHOOK: query: load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_part_stat partition (ds='2011') +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_part_stat partition (ds='2011') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@src_rc_merge_test_part_stat@ds=2011 +PREHOOK: query: show table extended like `src_rc_merge_test_part_stat` partition (ds='2011') +PREHOOK: type: SHOW_TABLESTATUS +POSTHOOK: query: show table extended like `src_rc_merge_test_part_stat` partition (ds='2011') +POSTHOOK: type: SHOW_TABLESTATUS +tableName:src_rc_merge_test_part_stat +owner:heyongqiang +location:pfile:/Users/heyongqiang/Documents/workspace/Hive-3/build/ql/test/data/warehouse/src_rc_merge_test_part_stat/ds=2011 +inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat +outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat +columns:struct columns { i32 key, string value} +partitioned:true +partitionColumns:struct partition_columns { string ds} +totalNumberFiles:3 +totalFileSize:636 +maxFileSize:222 +minFileSize:206 +lastAccessTime:0 +lastUpdateTime:1297290003000 + +PREHOOK: query: desc extended src_rc_merge_test_part_stat +PREHOOK: type: DESCTABLE +POSTHOOK: query: desc extended src_rc_merge_test_part_stat +POSTHOOK: type: DESCTABLE +key int from deserializer +value string from deserializer +ds string + +Detailed Table Information Table(tableName:src_rc_merge_test_part_stat, dbName:default, owner:heyongqiang, createTime:1297290001, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/heyongqiang/Documents/workspace/Hive-3/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=1297290001}, 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 +PREHOOK: Output: default@src_rc_merge_test_part_stat +PREHOOK: Output: default@src_rc_merge_test_part_stat@ds=2011 +POSTHOOK: query: analyze table src_rc_merge_test_part_stat partition(ds='2011') compute statistics +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_rc_merge_test_part_stat@ds=2011 +POSTHOOK: Output: default@src_rc_merge_test_part_stat +POSTHOOK: Output: default@src_rc_merge_test_part_stat@ds=2011 +PREHOOK: query: desc extended src_rc_merge_test_part_stat +PREHOOK: type: DESCTABLE +POSTHOOK: query: desc extended src_rc_merge_test_part_stat +POSTHOOK: type: DESCTABLE +key int from deserializer +value string from deserializer +ds string + +Detailed Table Information Table(tableName:src_rc_merge_test_part_stat, dbName:default, owner:heyongqiang, createTime:1297290001, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/heyongqiang/Documents/workspace/Hive-3/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=1297290011, 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 +PREHOOK: Output: default@src_rc_merge_test_part_stat@ds=2011 +POSTHOOK: query: alter table src_rc_merge_test_part_stat partition (ds='2011') concatenate +POSTHOOK: type: ALTER_PARTITION_MERGE +POSTHOOK: Input: default@src_rc_merge_test_part_stat +POSTHOOK: Output: default@src_rc_merge_test_part_stat@ds=2011 +PREHOOK: query: show table extended like `src_rc_merge_test_part_stat` partition (ds='2011') +PREHOOK: type: SHOW_TABLESTATUS +POSTHOOK: query: show table extended like `src_rc_merge_test_part_stat` partition (ds='2011') +POSTHOOK: type: SHOW_TABLESTATUS +tableName:src_rc_merge_test_part_stat +owner:heyongqiang +location:pfile:/Users/heyongqiang/Documents/workspace/Hive-3/build/ql/test/data/warehouse/src_rc_merge_test_part_stat/ds=2011 +inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat +outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat +columns:struct columns { i32 key, string value} +partitioned:true +partitionColumns:struct partition_columns { string ds} +totalNumberFiles:1 +totalFileSize:334 +maxFileSize:334 +minFileSize:334 +lastAccessTime:0 +lastUpdateTime:1297290013000 + +PREHOOK: query: desc extended src_rc_merge_test_part_stat +PREHOOK: type: DESCTABLE +POSTHOOK: query: desc extended src_rc_merge_test_part_stat +POSTHOOK: type: DESCTABLE +key int from deserializer +value string from deserializer +ds string + +Detailed Table Information Table(tableName:src_rc_merge_test_part_stat, dbName:default, owner:heyongqiang, createTime:1297290001, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/heyongqiang/Documents/workspace/Hive-3/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=1297290013, numRows=6, totalSize=334}, 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 +PREHOOK: Output: default@src_rc_merge_test_stat +POSTHOOK: query: drop table src_rc_merge_test_stat +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@src_rc_merge_test_stat +POSTHOOK: Output: default@src_rc_merge_test_stat +PREHOOK: query: drop table src_rc_merge_test_part_stat +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@src_rc_merge_test_part_stat +PREHOOK: Output: default@src_rc_merge_test_part_stat +POSTHOOK: query: drop table src_rc_merge_test_part_stat +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@src_rc_merge_test_part_stat +POSTHOOK: Output: default@src_rc_merge_test_part_stat Modified: hive/trunk/shims/src/0.20/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java URL: http://svn.apache.org/viewvc/hive/trunk/shims/src/0.20/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java?rev=1073992&r1=1073991&r2=1073992&view=diff ============================================================================== --- hive/trunk/shims/src/0.20/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java (original) +++ hive/trunk/shims/src/0.20/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java Wed Feb 23 23:58:41 2011 @@ -148,35 +148,6 @@ public class Hadoop20Shims implements Ha } } - public static class CombineHiveKey implements WritableComparable { - Object key; - - public CombineHiveKey(Object key) { - this.key = key; - } - - public Object getKey() { - return key; - } - - public void setKey(Object key) { - this.key = key; - } - - public void write(DataOutput out) throws IOException { - throw new IOException("Method not supported"); - } - - public void readFields(DataInput in) throws IOException { - throw new IOException("Method not supported"); - } - - public int compareTo(Object w) { - assert false; - return 0; - } - } - /* This class should be replaced with org.apache.hadoop.mapred.lib.CombineFileRecordReader class, once * https://issues.apache.org/jira/browse/MAPREDUCE-955 is fixed. This code should be removed - it is a copy * of org.apache.hadoop.mapred.lib.CombineFileRecordReader Modified: hive/trunk/shims/src/0.20S/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java URL: http://svn.apache.org/viewvc/hive/trunk/shims/src/0.20S/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java?rev=1073992&r1=1073991&r2=1073992&view=diff ============================================================================== --- hive/trunk/shims/src/0.20S/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java (original) +++ hive/trunk/shims/src/0.20S/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java Wed Feb 23 23:58:41 2011 @@ -150,35 +150,6 @@ public class Hadoop20SShims implements H } } - public static class CombineHiveKey implements WritableComparable { - Object key; - - public CombineHiveKey(Object key) { - this.key = key; - } - - public Object getKey() { - return key; - } - - public void setKey(Object key) { - this.key = key; - } - - public void write(DataOutput out) throws IOException { - throw new IOException("Method not supported"); - } - - public void readFields(DataInput in) throws IOException { - throw new IOException("Method not supported"); - } - - public int compareTo(Object w) { - assert false; - return 0; - } - } - /* This class should be replaced with org.apache.hadoop.mapred.lib.CombineFileRecordReader class, once * https://issues.apache.org/jira/browse/MAPREDUCE-955 is fixed. This code should be removed - it is a copy * of org.apache.hadoop.mapred.lib.CombineFileRecordReader Added: hive/trunk/shims/src/common/java/org/apache/hadoop/hive/shims/CombineHiveKey.java URL: http://svn.apache.org/viewvc/hive/trunk/shims/src/common/java/org/apache/hadoop/hive/shims/CombineHiveKey.java?rev=1073992&view=auto ============================================================================== --- hive/trunk/shims/src/common/java/org/apache/hadoop/hive/shims/CombineHiveKey.java (added) +++ hive/trunk/shims/src/common/java/org/apache/hadoop/hive/shims/CombineHiveKey.java Wed Feb 23 23:58:41 2011 @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.shims; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.io.WritableComparable; + +public class CombineHiveKey implements WritableComparable { + Object key; + + public CombineHiveKey(Object key) { + this.key = key; + } + + public Object getKey() { + return key; + } + + public void setKey(Object key) { + this.key = key; + } + + public void write(DataOutput out) throws IOException { + throw new IOException("Method not supported"); + } + + public void readFields(DataInput in) throws IOException { + throw new IOException("Method not supported"); + } + + public int compareTo(Object w) { + assert false; + return 0; + } +} \ No newline at end of file